From 11aece0c45d5b4fa38870d1208a3e782bfb627f8 Mon Sep 17 00:00:00 2001 From: Chris Twiner Date: Fri, 8 Mar 2024 10:51:31 +0100 Subject: [PATCH] #787 - ml internals removal - all public - #300 --- .../scala/frameless/FramelessInternals.scala | 15 +- .../main/scala/frameless/TypedDataset.scala | 2224 ++++++++++------- .../main/scala/frameless/TypedEncoder.scala | 8 +- .../functions/NonAggregateFunctions.scala | 1409 ++++++----- .../scala/frameless/functions/package.scala | 2 +- .../main/scala/frameless/ops/GroupByOps.scala | 6 +- .../scala/frameless/reflection/package.scala | 4 +- .../scala/frameless/UdtEncodedClass.scala | 27 +- ml/src/main/scala/frameless/ml/package.scala | 10 +- .../apache/spark/ml/FramelessInternals.scala | 13 - 10 files changed, 2160 insertions(+), 1558 deletions(-) delete mode 100644 ml/src/main/scala/org/apache/spark/ml/FramelessInternals.scala diff --git a/dataset/src/main/scala/frameless/FramelessInternals.scala b/dataset/src/main/scala/frameless/FramelessInternals.scala index 78684e7b1..b658788ea 100644 --- a/dataset/src/main/scala/frameless/FramelessInternals.scala +++ b/dataset/src/main/scala/frameless/FramelessInternals.scala @@ -1,11 +1,18 @@ package frameless -import com.sparkutils.shim.expressions.{Alias2 => Alias, CreateStruct1 => CreateStruct} -import org.apache.spark.sql.shim.{utils => shimUtils} +import com.sparkutils.shim.expressions.{ + Alias2 => Alias, + CreateStruct1 => CreateStruct +} +import org.apache.spark.sql.shim.{ utils => shimUtils } import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression, NonSQLExpression} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.catalyst.expressions.{ + Expression, + NamedExpression, + NonSQLExpression +} +import org.apache.spark.sql.catalyst.plans.logical.{ LogicalPlan, Project } import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.types._ import org.apache.spark.sql._ diff --git a/dataset/src/main/scala/frameless/TypedDataset.scala b/dataset/src/main/scala/frameless/TypedDataset.scala index 8a75c009e..0c31b5d16 100644 --- a/dataset/src/main/scala/frameless/TypedDataset.scala +++ b/dataset/src/main/scala/frameless/TypedDataset.scala @@ -4,36 +4,52 @@ import java.util import frameless.functions.CatalystExplodableCollection import frameless.ops._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Column, DataFrame, Dataset, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} +import org.apache.spark.sql.{ Column, DataFrame, Dataset, SparkSession } +import org.apache.spark.sql.catalyst.expressions.{ + Attribute, + AttributeReference, + Literal +} +import org.apache.spark.sql.catalyst.plans.logical.{ Join, JoinHint } import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.types.StructType import shapeless._ import shapeless.labelled.FieldType -import shapeless.ops.hlist.{Diff, IsHCons, Mapper, Prepend, ToTraversable, Tupler} -import shapeless.ops.record.{Keys, Modifier, Remover, Values} +import shapeless.ops.hlist.{ + Diff, + IsHCons, + Mapper, + Prepend, + ToTraversable, + Tupler +} +import shapeless.ops.record.{ Keys, Modifier, Remover, Values } import scala.language.experimental.macros -/** [[TypedDataset]] is a safer interface for working with `Dataset`. - * - * NOTE: Prefer `TypedDataset.create` over `new TypedDataset` unless you - * know what you are doing. - * - * Documentation marked "apache/spark" is thanks to apache/spark Contributors - * at https://github.com/apache/spark, licensed under Apache v2.0 available at - * http://www.apache.org/licenses/LICENSE-2.0 - */ -class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val encoder: TypedEncoder[T]) +/** + * [[TypedDataset]] is a safer interface for working with `Dataset`. + * + * NOTE: Prefer `TypedDataset.create` over `new TypedDataset` unless you + * know what you are doing. + * + * Documentation marked "apache/spark" is thanks to apache/spark Contributors + * at https://github.com/apache/spark, licensed under Apache v2.0 available at + * http://www.apache.org/licenses/LICENSE-2.0 + */ +class TypedDataset[T] protected[frameless] ( + val dataset: Dataset[T] + )(implicit + val encoder: TypedEncoder[T]) extends TypedDatasetForwarded[T] { self => private implicit val spark: SparkSession = dataset.sparkSession - /** Aggregates on the entire Dataset without groups. - * - * apache/spark - */ + /** + * Aggregates on the entire Dataset without groups. + * + * apache/spark + */ def agg[A](ca: TypedAggregate[T, A]): TypedDataset[A] = { implicit val ea = ca.uencoder val tuple1: TypedDataset[Tuple1[A]] = aggMany(ca) @@ -42,10 +58,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val TypedEncoder[A].catalystRepr match { case StructType(_) => // if column is struct, we use all its fields - val df = tuple1 - .dataset - .selectExpr("_1.*") - .as[A](TypedExpressionEncoder[A]) + val df = + tuple1.dataset.selectExpr("_1.*").as[A](TypedExpressionEncoder[A]) TypedDataset.create(df) case other => @@ -54,52 +68,59 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } } - /** Aggregates on the entire Dataset without groups. - * - * apache/spark - */ + /** + * Aggregates on the entire Dataset without groups. + * + * apache/spark + */ def agg[A, B]( - ca: TypedAggregate[T, A], - cb: TypedAggregate[T, B] - ): TypedDataset[(A, B)] = { + ca: TypedAggregate[T, A], + cb: TypedAggregate[T, B] + ): TypedDataset[(A, B)] = { implicit val (ea, eb) = (ca.uencoder, cb.uencoder) aggMany(ca, cb) } - /** Aggregates on the entire Dataset without groups. - * - * apache/spark - */ + /** + * Aggregates on the entire Dataset without groups. + * + * apache/spark + */ def agg[A, B, C]( - ca: TypedAggregate[T, A], - cb: TypedAggregate[T, B], - cc: TypedAggregate[T, C] - ): TypedDataset[(A, B, C)] = { + ca: TypedAggregate[T, A], + cb: TypedAggregate[T, B], + cc: TypedAggregate[T, C] + ): TypedDataset[(A, B, C)] = { implicit val (ea, eb, ec) = (ca.uencoder, cb.uencoder, cc.uencoder) aggMany(ca, cb, cc) } - /** Aggregates on the entire Dataset without groups. - * - * apache/spark - */ + /** + * Aggregates on the entire Dataset without groups. + * + * apache/spark + */ def agg[A, B, C, D]( - ca: TypedAggregate[T, A], - cb: TypedAggregate[T, B], - cc: TypedAggregate[T, C], - cd: TypedAggregate[T, D] - ): TypedDataset[(A, B, C, D)] = { - implicit val (ea, eb, ec, ed) = (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder) + ca: TypedAggregate[T, A], + cb: TypedAggregate[T, B], + cc: TypedAggregate[T, C], + cd: TypedAggregate[T, D] + ): TypedDataset[(A, B, C, D)] = { + implicit val (ea, eb, ec, ed) = + (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder) aggMany(ca, cb, cc, cd) } - /** Aggregates on the entire Dataset without groups. - * - * apache/spark - */ + /** + * Aggregates on the entire Dataset without groups. + * + * apache/spark + */ object aggMany extends ProductArgs { - def applyProduct[U <: HList, Out0 <: HList, Out](columns: U) - (implicit + + def applyProduct[U <: HList, Out0 <: HList, Out]( + columns: U + )(implicit i0: AggregateTypes.Aux[T, U, Out0], i1: ToTraversable.Aux[U, List, UntypedExpression[T]], i2: Tupler.Aux[Out0, Out], @@ -109,7 +130,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val val underlyingColumns = columns.toList[UntypedExpression[T]] val cols: Seq[Column] = for { (c, i) <- columns.toList[UntypedExpression[T]].zipWithIndex - } yield new Column(c.expr).as(s"_${i+1}") + } yield new Column(c.expr).as(s"_${i + 1}") // Workaround to SPARK-20346. One alternative is to allow the result to be Vector(null) for empty DataFrames. // Another one would be to return an Option. @@ -117,129 +138,163 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val for { (c, i) <- underlyingColumns.zipWithIndex if !c.uencoder.nullable - } yield s"_${i+1} is not null" - ).mkString(" or ") + } yield s"_${i + 1} is not null" + ).mkString(" or ") - val selected = dataset.toDF().agg(cols.head, cols.tail:_*).as[Out](TypedExpressionEncoder[Out]) - TypedDataset.create[Out](if (filterStr.isEmpty) selected else selected.filter(filterStr)) + val selected = dataset + .toDF() + .agg(cols.head, cols.tail: _*) + .as[Out](TypedExpressionEncoder[Out]) + TypedDataset.create[Out]( + if (filterStr.isEmpty) selected else selected.filter(filterStr) + ) } } /** Returns a new [[TypedDataset]] where each record has been mapped on to the specified type. */ - def as[U]()(implicit as: As[T, U]): TypedDataset[U] = { + def as[U]( + )(implicit + as: As[T, U] + ): TypedDataset[U] = { implicit val uencoder = as.encoder TypedDataset.create(dataset.as[U](TypedExpressionEncoder[U])) } - /** Returns a checkpointed version of this [[TypedDataset]]. Checkpointing can be used to truncate the - * logical plan of this Dataset, which is especially useful in iterative algorithms where the - * plan may grow exponentially. It will be saved to files inside the checkpoint - * directory set with `SparkContext#setCheckpointDir`. - * - * Differs from `Dataset#checkpoint` by wrapping its result into an effect-suspending `F[_]`. - * - * apache/spark - */ - def checkpoint[F[_]](eager: Boolean)(implicit F: SparkDelay[F]): F[TypedDataset[T]] = + /** + * Returns a checkpointed version of this [[TypedDataset]]. Checkpointing can be used to truncate the + * logical plan of this Dataset, which is especially useful in iterative algorithms where the + * plan may grow exponentially. It will be saved to files inside the checkpoint + * directory set with `SparkContext#setCheckpointDir`. + * + * Differs from `Dataset#checkpoint` by wrapping its result into an effect-suspending `F[_]`. + * + * apache/spark + */ + def checkpoint[F[_]]( + eager: Boolean + )(implicit + F: SparkDelay[F] + ): F[TypedDataset[T]] = F.delay(TypedDataset.create[T](dataset.checkpoint(eager))) - /** Returns a new [[TypedDataset]] where each record has been mapped on to the specified type. - * Unlike `as` the projection U may include a subset of the columns of T and the column names and types must agree. - * - * {{{ - * case class Foo(i: Int, j: String) - * case class Bar(j: String) - * - * val t: TypedDataset[Foo] = ... - * val b: TypedDataset[Bar] = t.project[Bar] - * - * case class BarErr(e: String) - * // The following does not compile because `Foo` doesn't have a field with name `e` - * val e: TypedDataset[BarErr] = t.project[BarErr] - * }}} - */ - def project[U](implicit projector: SmartProject[T,U]): TypedDataset[U] = projector.apply(this) - - /** Returns a new [[TypedDataset]] that contains the elements of both this and the `other` [[TypedDataset]] - * combined. - * - * Note that, this function is not a typical set union operation, in that it does not eliminate - * duplicate items. As such, it is analogous to `UNION ALL` in SQL. - * - * Differs from `Dataset#union` by aligning fields if possible. - * It will not compile if `Datasets` have not compatible schema. - * - * Example: - * {{{ - * case class Foo(x: Int, y: Long) - * case class Bar(y: Long, x: Int) - * case class Faz(x: Int, y: Int, z: Int) - * - * foo: TypedDataset[Foo] = ... - * bar: TypedDataset[Bar] = ... - * faz: TypedDataset[Faz] = ... - * - * foo union bar: TypedDataset[Foo] - * foo union faz: TypedDataset[Foo] - * // won't compile, you need to reverse order, you can't project from less fields to more - * faz union foo - * - * }}} - * - * apache/spark - */ - def union[U: TypedEncoder](other: TypedDataset[U])(implicit projector: SmartProject[U, T]): TypedDataset[T] = + /** + * Returns a new [[TypedDataset]] where each record has been mapped on to the specified type. + * Unlike `as` the projection U may include a subset of the columns of T and the column names and types must agree. + * + * {{{ + * case class Foo(i: Int, j: String) + * case class Bar(j: String) + * + * val t: TypedDataset[Foo] = ... + * val b: TypedDataset[Bar] = t.project[Bar] + * + * case class BarErr(e: String) + * // The following does not compile because `Foo` doesn't have a field with name `e` + * val e: TypedDataset[BarErr] = t.project[BarErr] + * }}} + */ + def project[U]( + implicit + projector: SmartProject[T, U] + ): TypedDataset[U] = projector.apply(this) + + /** + * Returns a new [[TypedDataset]] that contains the elements of both this and the `other` [[TypedDataset]] + * combined. + * + * Note that, this function is not a typical set union operation, in that it does not eliminate + * duplicate items. As such, it is analogous to `UNION ALL` in SQL. + * + * Differs from `Dataset#union` by aligning fields if possible. + * It will not compile if `Datasets` have not compatible schema. + * + * Example: + * {{{ + * case class Foo(x: Int, y: Long) + * case class Bar(y: Long, x: Int) + * case class Faz(x: Int, y: Int, z: Int) + * + * foo: TypedDataset[Foo] = ... + * bar: TypedDataset[Bar] = ... + * faz: TypedDataset[Faz] = ... + * + * foo union bar: TypedDataset[Foo] + * foo union faz: TypedDataset[Foo] + * // won't compile, you need to reverse order, you can't project from less fields to more + * faz union foo + * + * }}} + * + * apache/spark + */ + def union[U: TypedEncoder]( + other: TypedDataset[U] + )(implicit + projector: SmartProject[U, T] + ): TypedDataset[T] = TypedDataset.create(dataset.union(other.project[T].dataset)) - /** Returns a new [[TypedDataset]] that contains the elements of both this and the `other` [[TypedDataset]] - * combined. - * - * Note that, this function is not a typical set union operation, in that it does not eliminate - * duplicate items. As such, it is analogous to `UNION ALL` in SQL. - * - * apache/spark - */ + /** + * Returns a new [[TypedDataset]] that contains the elements of both this and the `other` [[TypedDataset]] + * combined. + * + * Note that, this function is not a typical set union operation, in that it does not eliminate + * duplicate items. As such, it is analogous to `UNION ALL` in SQL. + * + * apache/spark + */ def union(other: TypedDataset[T]): TypedDataset[T] = { TypedDataset.create(dataset.union(other.dataset)) } - /** Returns the number of elements in the [[TypedDataset]]. - * - * Differs from `Dataset#count` by wrapping its result into an effect-suspending `F[_]`. - */ - def count[F[_]]()(implicit F: SparkDelay[F]): F[Long] = + /** + * Returns the number of elements in the [[TypedDataset]]. + * + * Differs from `Dataset#count` by wrapping its result into an effect-suspending `F[_]`. + */ + def count[F[_]]( + )(implicit + F: SparkDelay[F] + ): F[Long] = F.delay(dataset.count()) - /** Returns `TypedColumn` of type `A` given its name (alias for `col`). - * - * {{{ - * tf('id) - * }}} - * - * It is statically checked that column with such name exists and has type `A`. - */ - def apply[A](column: Witness.Lt[Symbol]) - (implicit + /** + * Returns `TypedColumn` of type `A` given its name (alias for `col`). + * + * {{{ + * tf('id) + * }}} + * + * It is statically checked that column with such name exists and has type `A`. + */ + def apply[A]( + column: Witness.Lt[Symbol] + )(implicit i0: TypedColumn.Exists[T, column.T, A], i1: TypedEncoder[A] ): TypedColumn[T, A] = col(column) - /** Returns `TypedColumn` of type `A` given its name. - * - * {{{ - * tf.col('id) - * }}} - * - * It is statically checked that column with such name exists and has type `A`. - */ - def col[A](column: Witness.Lt[Symbol]) - (implicit + /** + * Returns `TypedColumn` of type `A` given its name. + * + * {{{ + * tf.col('id) + * }}} + * + * It is statically checked that column with such name exists and has type `A`. + */ + def col[A]( + column: Witness.Lt[Symbol] + )(implicit i0: TypedColumn.Exists[T, column.T, A], i1: TypedEncoder[A] ): TypedColumn[T, A] = - new TypedColumn[T, A](dataset(column.value.name).as[A](TypedExpressionEncoder[A])) + new TypedColumn[T, A]( + dataset(column.value.name).as[A](TypedExpressionEncoder[A]) + ) - /** Returns `TypedColumn` of type `A` given a lambda indicating the field. + /** + * Returns `TypedColumn` of type `A` given a lambda indicating the field. * * {{{ * td.col(_.id) @@ -250,12 +305,13 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val def col[A](x: Function1[T, A]): TypedColumn[T, A] = macro TypedColumnMacroImpl.applyImpl[T, A] - /** Projects the entire `TypedDataset[T]` into a single column of type `TypedColumn[T,T]`. - * {{{ - * ts: TypedDataset[Foo] = ... - * ts.select(ts.asCol, ts.asCol): TypedDataset[(Foo,Foo)] - * }}} - */ + /** + * Projects the entire `TypedDataset[T]` into a single column of type `TypedColumn[T,T]`. + * {{{ + * ts: TypedDataset[Foo] = ... + * ts.select(ts.asCol, ts.asCol): TypedDataset[(Foo,Foo)] + * }}} + */ def asCol: TypedColumn[T, T] = { val projectedColumn: Column = encoder.catalystRepr match { case StructType(_) => @@ -265,78 +321,98 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val case _ => dataset.col(dataset.columns.head) } - - new TypedColumn[T,T](projectedColumn) + + new TypedColumn[T, T](projectedColumn) } - /** References the entire `TypedDataset[T]` as a single column - * of type `TypedColumn[T,T]` so it can be used in a join operation. - * - * {{{ - * def nameJoin(ds1: TypedDataset[Person], ds2: TypedDataset[Name]) = - * ds1.joinLeftSemi(ds2)(ds1.col('name) === ds2.asJoinColValue) - * }}} - */ - def asJoinColValue(implicit i0: IsValueClass[T]): TypedColumn[T, T] = { + /** + * References the entire `TypedDataset[T]` as a single column + * of type `TypedColumn[T,T]` so it can be used in a join operation. + * + * {{{ + * def nameJoin(ds1: TypedDataset[Person], ds2: TypedDataset[Name]) = + * ds1.joinLeftSemi(ds2)(ds1.col('name) === ds2.asJoinColValue) + * }}} + */ + def asJoinColValue( + implicit + i0: IsValueClass[T] + ): TypedColumn[T, T] = { import _root_.frameless.syntax._ dataset.col("value").typedColumn } object colMany extends SingletonProductArgs { - def applyProduct[U <: HList, Out](columns: U) - (implicit + + def applyProduct[U <: HList, Out]( + columns: U + )(implicit i0: TypedColumn.ExistsMany[T, U, Out], i1: TypedEncoder[Out], i2: ToTraversable.Aux[U, List, Symbol] ): TypedColumn[T, Out] = { - val names = columns.toList[Symbol].map(_.name) - val colExpr = FramelessInternals.resolveExpr(dataset, names) - new TypedColumn[T, Out](colExpr) - } + val names = columns.toList[Symbol].map(_.name) + val colExpr = FramelessInternals.resolveExpr(dataset, names) + new TypedColumn[T, Out](colExpr) + } } - /** Right hand side disambiguation of `col` for join expressions. - * To be used when writting self-joins, noop in other circumstances. - * - * Note: In vanilla Spark, disambiguation in self-joins is acheaved using - * String based aliases, which is obviously unsafe. - */ - def colRight[A](column: Witness.Lt[Symbol]) - (implicit + /** + * Right hand side disambiguation of `col` for join expressions. + * To be used when writting self-joins, noop in other circumstances. + * + * Note: In vanilla Spark, disambiguation in self-joins is acheaved using + * String based aliases, which is obviously unsafe. + */ + def colRight[A]( + column: Witness.Lt[Symbol] + )(implicit i0: TypedColumn.Exists[T, column.T, A], i1: TypedEncoder[A] ): TypedColumn[T, A] = - new TypedColumn[T, A](FramelessInternals.DisambiguateRight(col(column).expr)) - - /** Left hand side disambiguation of `col` for join expressions. - * To be used when writting self-joins, noop in other circumstances. - * - * Note: In vanilla Spark, disambiguation in self-joins is acheaved using - * String based aliases, which is obviously unsafe. - */ - def colLeft[A](column: Witness.Lt[Symbol]) - (implicit + new TypedColumn[T, A]( + FramelessInternals.DisambiguateRight(col(column).expr) + ) + + /** + * Left hand side disambiguation of `col` for join expressions. + * To be used when writting self-joins, noop in other circumstances. + * + * Note: In vanilla Spark, disambiguation in self-joins is acheaved using + * String based aliases, which is obviously unsafe. + */ + def colLeft[A]( + column: Witness.Lt[Symbol] + )(implicit i0: TypedColumn.Exists[T, column.T, A], i1: TypedEncoder[A] ): TypedColumn[T, A] = - new TypedColumn[T, A](FramelessInternals.DisambiguateLeft(col(column).expr)) - - /** Returns a `Seq` that contains all the elements in this [[TypedDataset]]. - * - * Running this operation requires moving all the data into the application's driver process, and - * doing so on a very large [[TypedDataset]] can crash the driver process with OutOfMemoryError. - * - * Differs from `Dataset#collect` by wrapping its result into an effect-suspending `F[_]`. - */ - def collect[F[_]]()(implicit F: SparkDelay[F]): F[Seq[T]] = + new TypedColumn[T, A](FramelessInternals.DisambiguateLeft(col(column).expr)) + + /** + * Returns a `Seq` that contains all the elements in this [[TypedDataset]]. + * + * Running this operation requires moving all the data into the application's driver process, and + * doing so on a very large [[TypedDataset]] can crash the driver process with OutOfMemoryError. + * + * Differs from `Dataset#collect` by wrapping its result into an effect-suspending `F[_]`. + */ + def collect[F[_]]( + )(implicit + F: SparkDelay[F] + ): F[Seq[T]] = F.delay(dataset.collect().toSeq) - /** Optionally returns the first element in this [[TypedDataset]]. - * - * Differs from `Dataset#first` by wrapping its result into an `Option` and an effect-suspending `F[_]`. - */ - def firstOption[F[_]]()(implicit F: SparkDelay[F]): F[Option[T]] = + /** + * Optionally returns the first element in this [[TypedDataset]]. + * + * Differs from `Dataset#first` by wrapping its result into an `Option` and an effect-suspending `F[_]`. + */ + def firstOption[F[_]]( + )(implicit + F: SparkDelay[F] + ): F[Option[T]] = F.delay { try { Option(dataset.first()) @@ -345,354 +421,462 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } } - /** Returns the first `num` elements of this [[TypedDataset]] as a `Seq`. - * - * Running take requires moving data into the application's driver process, and doing so with - * a very large `num` can crash the driver process with OutOfMemoryError. - * - * Differs from `Dataset#take` by wrapping its result into an effect-suspending `F[_]`. - * - * apache/spark - */ - def take[F[_]](num: Int)(implicit F: SparkDelay[F]): F[Seq[T]] = + /** + * Returns the first `num` elements of this [[TypedDataset]] as a `Seq`. + * + * Running take requires moving data into the application's driver process, and doing so with + * a very large `num` can crash the driver process with OutOfMemoryError. + * + * Differs from `Dataset#take` by wrapping its result into an effect-suspending `F[_]`. + * + * apache/spark + */ + def take[F[_]]( + num: Int + )(implicit + F: SparkDelay[F] + ): F[Seq[T]] = F.delay(dataset.take(num).toSeq) - /** Return an iterator that contains all rows in this [[TypedDataset]]. - * - * The iterator will consume as much memory as the largest partition in this [[TypedDataset]]. - * - * NOTE: this results in multiple Spark jobs, and if the input [[TypedDataset]] is the result - * of a wide transformation (e.g. join with different partitioners), to avoid - * recomputing the input [[TypedDataset]] should be cached first. - * - * Differs from `Dataset#toLocalIterator()` by wrapping its result into an effect-suspending `F[_]`. - * - * apache/spark - */ - def toLocalIterator[F[_]]()(implicit F: SparkDelay[F]): F[util.Iterator[T]] = + /** + * Return an iterator that contains all rows in this [[TypedDataset]]. + * + * The iterator will consume as much memory as the largest partition in this [[TypedDataset]]. + * + * NOTE: this results in multiple Spark jobs, and if the input [[TypedDataset]] is the result + * of a wide transformation (e.g. join with different partitioners), to avoid + * recomputing the input [[TypedDataset]] should be cached first. + * + * Differs from `Dataset#toLocalIterator()` by wrapping its result into an effect-suspending `F[_]`. + * + * apache/spark + */ + def toLocalIterator[F[_]]( + )(implicit + F: SparkDelay[F] + ): F[util.Iterator[T]] = F.delay(dataset.toLocalIterator()) - /** Alias for firstOption(). - */ - def headOption[F[_]]()(implicit F: SparkDelay[F]): F[Option[T]] = firstOption() + /** + * Alias for firstOption(). + */ + def headOption[F[_]]( + )(implicit + F: SparkDelay[F] + ): F[Option[T]] = firstOption() - /** Alias for take(). - */ - def head[F[_]](num: Int)(implicit F: SparkDelay[F]): F[Seq[T]] = take(num) + /** + * Alias for take(). + */ + def head[F[_]]( + num: Int + )(implicit + F: SparkDelay[F] + ): F[Seq[T]] = take(num) // $COVERAGE-OFF$ - /** Alias for firstOption(). - */ - @deprecated("Method may throw exception. Use headOption or firstOption instead.", "0.5.0") + /** + * Alias for firstOption(). + */ + @deprecated( + "Method may throw exception. Use headOption or firstOption instead.", + "0.5.0" + ) def head: T = dataset.head() - /** Alias for firstOption(). - */ - @deprecated("Method may throw exception. Use headOption or firstOption instead.", "0.5.0") + /** + * Alias for firstOption(). + */ + @deprecated( + "Method may throw exception. Use headOption or firstOption instead.", + "0.5.0" + ) def first: T = dataset.head() // $COVERAGE-ONN$ - /** Displays the content of this [[TypedDataset]] in a tabular form. Strings more than 20 characters - * will be truncated, and all cells will be aligned right. For example: - * {{{ - * year month AVG('Adj Close) MAX('Adj Close) - * 1980 12 0.503218 0.595103 - * 1981 01 0.523289 0.570307 - * 1982 02 0.436504 0.475256 - * 1983 03 0.410516 0.442194 - * 1984 04 0.450090 0.483521 - * }}} - * @param numRows Number of rows to show - * @param truncate Whether truncate long strings. If true, strings more than 20 characters will - * be truncated and all cells will be aligned right - * - * Differs from `Dataset#show` by wrapping its result into an effect-suspending `F[_]`. - * - * apache/spark - */ - def show[F[_]](numRows: Int = 20, truncate: Boolean = true)(implicit F: SparkDelay[F]): F[Unit] = + /** + * Displays the content of this [[TypedDataset]] in a tabular form. Strings more than 20 characters + * will be truncated, and all cells will be aligned right. For example: + * {{{ + * year month AVG('Adj Close) MAX('Adj Close) + * 1980 12 0.503218 0.595103 + * 1981 01 0.523289 0.570307 + * 1982 02 0.436504 0.475256 + * 1983 03 0.410516 0.442194 + * 1984 04 0.450090 0.483521 + * }}} + * @param numRows Number of rows to show + * @param truncate Whether truncate long strings. If true, strings more than 20 characters will + * be truncated and all cells will be aligned right + * + * Differs from `Dataset#show` by wrapping its result into an effect-suspending `F[_]`. + * + * apache/spark + */ + def show[F[_]]( + numRows: Int = 20, + truncate: Boolean = true + )(implicit + F: SparkDelay[F] + ): F[Unit] = F.delay(dataset.show(numRows, truncate)) - /** Returns a new [[frameless.TypedDataset]] that only contains elements where `column` is `true`. - * - * Differs from `TypedDatasetForward#filter` by taking a `TypedColumn[T, Boolean]` instead of a - * `T => Boolean`. Using a column expression instead of a regular function save one Spark → Scala - * deserialization which leads to better performance. - */ + /** + * Returns a new [[frameless.TypedDataset]] that only contains elements where `column` is `true`. + * + * Differs from `TypedDatasetForward#filter` by taking a `TypedColumn[T, Boolean]` instead of a + * `T => Boolean`. Using a column expression instead of a regular function save one Spark → Scala + * deserialization which leads to better performance. + */ def filter(column: TypedColumn[T, Boolean]): TypedDataset[T] = { - val filtered = dataset.toDF() - .filter(column.untyped) - .as[T](TypedExpressionEncoder[T]) + val filtered = + dataset.toDF().filter(column.untyped).as[T](TypedExpressionEncoder[T]) TypedDataset.create[T](filtered) } - /** Runs `func` on each element of this [[TypedDataset]]. - * - * Differs from `Dataset#foreach` by wrapping its result into an effect-suspending `F[_]`. - */ - def foreach[F[_]](func: T => Unit)(implicit F: SparkDelay[F]): F[Unit] = + /** + * Runs `func` on each element of this [[TypedDataset]]. + * + * Differs from `Dataset#foreach` by wrapping its result into an effect-suspending `F[_]`. + */ + def foreach[F[_]]( + func: T => Unit + )(implicit + F: SparkDelay[F] + ): F[Unit] = F.delay(dataset.foreach(func)) - /** Runs `func` on each partition of this [[TypedDataset]]. - * - * Differs from `Dataset#foreachPartition` by wrapping its result into an effect-suspending `F[_]`. - */ - def foreachPartition[F[_]](func: Iterator[T] => Unit)(implicit F: SparkDelay[F]): F[Unit] = + /** + * Runs `func` on each partition of this [[TypedDataset]]. + * + * Differs from `Dataset#foreachPartition` by wrapping its result into an effect-suspending `F[_]`. + */ + def foreachPartition[F[_]]( + func: Iterator[T] => Unit + )(implicit + F: SparkDelay[F] + ): F[Unit] = F.delay(dataset.foreachPartition(func)) /** - * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified column, - * so we can run aggregation on it. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified column, + * so we can run aggregation on it. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ def cube[K1]( - c1: TypedColumn[T, K1] - ): Cube1Ops[K1, T] = new Cube1Ops[K1, T](this, c1) - - /** - * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified columns, - * so we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + c1: TypedColumn[T, K1] + ): Cube1Ops[K1, T] = new Cube1Ops[K1, T](this, c1) + + /** + * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified columns, + * so we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ def cube[K1, K2]( - c1: TypedColumn[T, K1], - c2: TypedColumn[T, K2] - ): Cube2Ops[K1, K2, T] = new Cube2Ops[K1, K2, T](this, c1, c2) - - /** - * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified columns, - * so we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * {{{ - * case class MyClass(a: Int, b: Int, c: Int) - * val ds: TypedDataset[MyClass] - - * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = - * ds.cubeMany(ds('a), ds('b)).agg(count[MyClass]()) - * - * // original dataset: - * a b c - * 10 20 1 - * 15 25 2 - * - * // after aggregation: - * _1 _2 _3 - * 15 null 1 - * 15 25 1 - * null null 2 - * null 25 1 - * null 20 1 - * 10 null 1 - * 10 20 1 - * - * }}} - * - * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + c1: TypedColumn[T, K1], + c2: TypedColumn[T, K2] + ): Cube2Ops[K1, K2, T] = new Cube2Ops[K1, K2, T](this, c1, c2) + + /** + * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified columns, + * so we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * {{{ + * case class MyClass(a: Int, b: Int, c: Int) + * val ds: TypedDataset[MyClass] + * + * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = + * ds.cubeMany(ds('a), ds('b)).agg(count[MyClass]()) + * + * // original dataset: + * a b c + * 10 20 1 + * 15 25 2 + * + * // after aggregation: + * _1 _2 _3 + * 15 null 1 + * 15 25 1 + * null null 2 + * null 25 1 + * null 20 1 + * 10 null 1 + * 10 20 1 + * + * }}} + * + * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ object cubeMany extends ProductArgs { - def applyProduct[TK <: HList, K <: HList, KT](groupedBy: TK) - (implicit + + def applyProduct[TK <: HList, K <: HList, KT]( + groupedBy: TK + )(implicit i0: ColumnTypes.Aux[T, TK, K], i1: Tupler.Aux[K, KT], i2: ToTraversable.Aux[TK, List, UntypedExpression[T]] - ): CubeManyOps[T, TK, K, KT] = new CubeManyOps[T, TK, K, KT](self, groupedBy) + ): CubeManyOps[T, TK, K, KT] = + new CubeManyOps[T, TK, K, KT](self, groupedBy) } /** - * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * apache/spark - */ + * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * apache/spark + */ def groupBy[K1]( - c1: TypedColumn[T, K1] - ): GroupedBy1Ops[K1, T] = new GroupedBy1Ops[K1, T](this, c1) + c1: TypedColumn[T, K1] + ): GroupedBy1Ops[K1, T] = new GroupedBy1Ops[K1, T](this, c1) /** - * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * apache/spark - */ + * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * apache/spark + */ def groupBy[K1, K2]( - c1: TypedColumn[T, K1], - c2: TypedColumn[T, K2] - ): GroupedBy2Ops[K1, K2, T] = new GroupedBy2Ops[K1, K2, T](this, c1, c2) - - /** - * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * {{{ - * case class MyClass(a: Int, b: Int, c: Int) - * val ds: TypedDataset[MyClass] - * - * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = - * ds.groupByMany(ds('a), ds('b)).agg(count[MyClass]()) - * - * // original dataset: - * a b c - * 10 20 1 - * 15 25 2 - * - * // after aggregation: - * _1 _2 _3 - * 10 20 1 - * 15 25 1 - * - * }}} - * - * apache/spark - */ + c1: TypedColumn[T, K1], + c2: TypedColumn[T, K2] + ): GroupedBy2Ops[K1, K2, T] = new GroupedBy2Ops[K1, K2, T](this, c1, c2) + + /** + * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * {{{ + * case class MyClass(a: Int, b: Int, c: Int) + * val ds: TypedDataset[MyClass] + * + * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = + * ds.groupByMany(ds('a), ds('b)).agg(count[MyClass]()) + * + * // original dataset: + * a b c + * 10 20 1 + * 15 25 2 + * + * // after aggregation: + * _1 _2 _3 + * 10 20 1 + * 15 25 1 + * + * }}} + * + * apache/spark + */ object groupByMany extends ProductArgs { - def applyProduct[TK <: HList, K <: HList, KT](groupedBy: TK) - (implicit + + def applyProduct[TK <: HList, K <: HList, KT]( + groupedBy: TK + )(implicit i0: ColumnTypes.Aux[T, TK, K], i1: Tupler.Aux[K, KT], i2: ToTraversable.Aux[TK, List, UntypedExpression[T]] - ): GroupedByManyOps[T, TK, K, KT] = new GroupedByManyOps[T, TK, K, KT](self, groupedBy) + ): GroupedByManyOps[T, TK, K, KT] = + new GroupedByManyOps[T, TK, K, KT](self, groupedBy) } /** - * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified column, - * so we can run aggregation on it. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified column, + * so we can run aggregation on it. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ def rollup[K1]( - c1: TypedColumn[T, K1] - ): Rollup1Ops[K1, T] = new Rollup1Ops[K1, T](this, c1) - - /** - * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified columns, - * so we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + c1: TypedColumn[T, K1] + ): Rollup1Ops[K1, T] = new Rollup1Ops[K1, T](this, c1) + + /** + * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified columns, + * so we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ def rollup[K1, K2]( - c1: TypedColumn[T, K1], - c2: TypedColumn[T, K2] - ): Rollup2Ops[K1, K2, T] = new Rollup2Ops[K1, K2, T](this, c1, c2) - - /** - * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified columns, - * so we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * {{{ - * case class MyClass(a: Int, b: Int, c: Int) - * val ds: TypedDataset[MyClass] - * - * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = - * ds.rollupMany(ds('a), ds('b)).agg(count[MyClass]()) - * - * // original dataset: - * a b c - * 10 20 1 - * 15 25 2 - * - * // after aggregation: - * _1 _2 _3 - * 15 null 1 - * 15 25 1 - * null null 2 - * 10 null 1 - * 10 20 1 - * - * }}} - * - * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + c1: TypedColumn[T, K1], + c2: TypedColumn[T, K2] + ): Rollup2Ops[K1, K2, T] = new Rollup2Ops[K1, K2, T](this, c1, c2) + + /** + * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified columns, + * so we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * {{{ + * case class MyClass(a: Int, b: Int, c: Int) + * val ds: TypedDataset[MyClass] + * + * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = + * ds.rollupMany(ds('a), ds('b)).agg(count[MyClass]()) + * + * // original dataset: + * a b c + * 10 20 1 + * 15 25 2 + * + * // after aggregation: + * _1 _2 _3 + * 15 null 1 + * 15 25 1 + * null null 2 + * 10 null 1 + * 10 20 1 + * + * }}} + * + * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ object rollupMany extends ProductArgs { - def applyProduct[TK <: HList, K <: HList, KT](groupedBy: TK) - (implicit + + def applyProduct[TK <: HList, K <: HList, KT]( + groupedBy: TK + )(implicit i0: ColumnTypes.Aux[T, TK, K], i1: Tupler.Aux[K, KT], i2: ToTraversable.Aux[TK, List, UntypedExpression[T]] - ): RollupManyOps[T, TK, K, KT] = new RollupManyOps[T, TK, K, KT](self, groupedBy) + ): RollupManyOps[T, TK, K, KT] = + new RollupManyOps[T, TK, K, KT](self, groupedBy) } /** Computes the cartesian project of `this` `Dataset` with the `other` `Dataset` */ - def joinCross[U](other: TypedDataset[U]) - (implicit e: TypedEncoder[(T, U)]): TypedDataset[(T, U)] = - new TypedDataset(self.dataset.joinWith(other.dataset, new Column(Literal(true)), "cross")) - - /** Computes the full outer join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinFull[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]) - (implicit e: TypedEncoder[(Option[T], Option[U])]): TypedDataset[(Option[T], Option[U])] = - new TypedDataset(self.dataset.joinWith(other.dataset, condition.untyped, "full") - .as[(Option[T], Option[U])](TypedExpressionEncoder[(Option[T], Option[U])])) - - /** Computes the inner join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinInner[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]) - (implicit e: TypedEncoder[(T, U)]): TypedDataset[(T, U)] = { - import FramelessInternals._ - - val leftPlan = logicalPlan(dataset) - val rightPlan = logicalPlan(other.dataset) - val join = disambiguate(Join(leftPlan, rightPlan, Inner, Some(condition.expr), JoinHint.NONE)) - val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) - val joinedDs = mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(T, U)]) - - TypedDataset.create[(T, U)](joinedDs) - } + def joinCross[U]( + other: TypedDataset[U] + )(implicit + e: TypedEncoder[(T, U)] + ): TypedDataset[(T, U)] = + new TypedDataset( + self.dataset.joinWith(other.dataset, new Column(Literal(true)), "cross") + ) - /** Computes the left outer join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinLeft[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]) - (implicit e: TypedEncoder[(T, Option[U])]): TypedDataset[(T, Option[U])] = - new TypedDataset(self.dataset.joinWith(other.dataset, condition.untyped, "left_outer") - .as[(T, Option[U])](TypedExpressionEncoder[(T, Option[U])])) - - /** Computes the left semi join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinLeftSemi[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]): TypedDataset[T] = - new TypedDataset(self.dataset.join(other.dataset, condition.untyped, "leftsemi") - .as[T](TypedExpressionEncoder(encoder))) - - /** Computes the left anti join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinLeftAnti[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]): TypedDataset[T] = - new TypedDataset(self.dataset.join(other.dataset, condition.untyped, "leftanti") - .as[T](TypedExpressionEncoder(encoder))) - - /** Computes the right outer join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinRight[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]) - (implicit e: TypedEncoder[(Option[T], U)]): TypedDataset[(Option[T], U)] = - new TypedDataset(self.dataset.joinWith(other.dataset, condition.untyped, "right_outer") - .as[(Option[T], U)](TypedExpressionEncoder[(Option[T], U)])) + /** + * Computes the full outer join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinFull[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + )(implicit + e: TypedEncoder[(Option[T], Option[U])] + ): TypedDataset[(Option[T], Option[U])] = + new TypedDataset( + self.dataset + .joinWith(other.dataset, condition.untyped, "full") + .as[(Option[T], Option[U])]( + TypedExpressionEncoder[(Option[T], Option[U])] + ) + ) + + /** + * Computes the inner join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinInner[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + )(implicit + e: TypedEncoder[(T, U)] + ): TypedDataset[(T, U)] = { + import FramelessInternals._ + + val leftPlan = logicalPlan(dataset) + val rightPlan = logicalPlan(other.dataset) + val join = disambiguate( + Join(leftPlan, rightPlan, Inner, Some(condition.expr), JoinHint.NONE) + ) + val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) + val joinedDs = + mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(T, U)]) + + TypedDataset.create[(T, U)](joinedDs) + } + + /** + * Computes the left outer join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinLeft[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + )(implicit + e: TypedEncoder[(T, Option[U])] + ): TypedDataset[(T, Option[U])] = + new TypedDataset( + self.dataset + .joinWith(other.dataset, condition.untyped, "left_outer") + .as[(T, Option[U])](TypedExpressionEncoder[(T, Option[U])]) + ) + + /** + * Computes the left semi join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinLeftSemi[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + ): TypedDataset[T] = + new TypedDataset( + self.dataset + .join(other.dataset, condition.untyped, "leftsemi") + .as[T](TypedExpressionEncoder(encoder)) + ) + + /** + * Computes the left anti join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinLeftAnti[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + ): TypedDataset[T] = + new TypedDataset( + self.dataset + .join(other.dataset, condition.untyped, "leftanti") + .as[T](TypedExpressionEncoder(encoder)) + ) + + /** + * Computes the right outer join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinRight[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + )(implicit + e: TypedEncoder[(Option[T], U)] + ): TypedDataset[(Option[T], U)] = + new TypedDataset( + self.dataset + .joinWith(other.dataset, condition.untyped, "right_outer") + .as[(Option[T], U)](TypedExpressionEncoder[(Option[T], U)]) + ) private def disambiguate(join: Join): Join = { - val plan = FramelessInternals.ofRows(dataset.sparkSession, join).queryExecution.analyzed.asInstanceOf[Join] + val plan = FramelessInternals + .ofRows(dataset.sparkSession, join) + .queryExecution + .analyzed + .asInstanceOf[Join] val disambiguated = plan.condition.map(_.transform { case FramelessInternals.DisambiguateLeft(tagged: AttributeReference) => val leftDs = FramelessInternals.ofRows(spark, plan.left) @@ -707,43 +891,81 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val plan.copy(condition = disambiguated) } - /** Takes a function from A => R and converts it to a UDF for TypedColumn[T, A] => TypedColumn[T, R]. - */ - def makeUDF[A: TypedEncoder, R: TypedEncoder](f: A => R): - TypedColumn[T, A] => TypedColumn[T, R] = functions.udf(f) - - /** Takes a function from (A1, A2) => R and converts it to a UDF for - * (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R]. - */ - def makeUDF[A1: TypedEncoder, A2: TypedEncoder, R: TypedEncoder](f: (A1, A2) => R): - (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R] = functions.udf(f) - - /** Takes a function from (A1, A2, A3) => R and converts it to a UDF for - * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3]) => TypedColumn[T, R]. - */ - def makeUDF[A1: TypedEncoder, A2: TypedEncoder, A3: TypedEncoder, R: TypedEncoder](f: (A1, A2, A3) => R): - (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3]) => TypedColumn[T, R] = functions.udf(f) - - /** Takes a function from (A1, A2, A3, A4) => R and converts it to a UDF for - * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4]) => TypedColumn[T, R]. - */ - def makeUDF[A1: TypedEncoder, A2: TypedEncoder, A3: TypedEncoder, A4: TypedEncoder, R: TypedEncoder](f: (A1, A2, A3, A4) => R): - (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4]) => TypedColumn[T, R] = functions.udf(f) - - /** Takes a function from (A1, A2, A3, A4, A5) => R and converts it to a UDF for - * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4], TypedColumn[T, A5]) => TypedColumn[T, R]. - */ - def makeUDF[A1: TypedEncoder, A2: TypedEncoder, A3: TypedEncoder, A4: TypedEncoder, A5: TypedEncoder, R: TypedEncoder](f: (A1, A2, A3, A4, A5) => R): - (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4], TypedColumn[T, A5]) => TypedColumn[T, R] = functions.udf(f) - - /** Type-safe projection from type T to Tuple1[A] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Takes a function from A => R and converts it to a UDF for TypedColumn[T, A] => TypedColumn[T, R]. + */ + def makeUDF[A: TypedEncoder, R: TypedEncoder](f: A => R): TypedColumn[T, A] => TypedColumn[T, R] = + functions.udf(f) + + /** + * Takes a function from (A1, A2) => R and converts it to a UDF for + * (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R]. + */ + def makeUDF[A1: TypedEncoder, A2: TypedEncoder, R: TypedEncoder]( + f: (A1, A2) => R + ): (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R] = + functions.udf(f) + + /** + * Takes a function from (A1, A2, A3) => R and converts it to a UDF for + * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3]) => TypedColumn[T, R]. + */ + def makeUDF[ + A1: TypedEncoder, + A2: TypedEncoder, + A3: TypedEncoder, + R: TypedEncoder + ](f: (A1, A2, A3) => R + ): (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3]) => TypedColumn[T, R] = + functions.udf(f) + + /** + * Takes a function from (A1, A2, A3, A4) => R and converts it to a UDF for + * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4]) => TypedColumn[T, R]. + */ + def makeUDF[ + A1: TypedEncoder, + A2: TypedEncoder, + A3: TypedEncoder, + A4: TypedEncoder, + R: TypedEncoder + ](f: (A1, A2, A3, A4) => R + ): ( + TypedColumn[T, A1], + TypedColumn[T, A2], + TypedColumn[T, A3], + TypedColumn[T, A4] + ) => TypedColumn[T, R] = functions.udf(f) + + /** + * Takes a function from (A1, A2, A3, A4, A5) => R and converts it to a UDF for + * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4], TypedColumn[T, A5]) => TypedColumn[T, R]. + */ + def makeUDF[ + A1: TypedEncoder, + A2: TypedEncoder, + A3: TypedEncoder, + A4: TypedEncoder, + A5: TypedEncoder, + R: TypedEncoder + ](f: (A1, A2, A3, A4, A5) => R + ): ( + TypedColumn[T, A1], + TypedColumn[T, A2], + TypedColumn[T, A3], + TypedColumn[T, A4], + TypedColumn[T, A5] + ) => TypedColumn[T, R] = functions.udf(f) + + /** + * Type-safe projection from type T to Tuple1[A] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A]( - ca: TypedColumn[T, A] - ): TypedDataset[A] = { + ca: TypedColumn[T, A] + ): TypedDataset[A] = { implicit val ea = ca.uencoder val tuple1: TypedDataset[Tuple1[A]] = selectMany(ca) @@ -753,10 +975,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val TypedEncoder[A].catalystRepr match { case StructType(_) => // if column is struct, we use all its fields - val df = tuple1 - .dataset - .selectExpr("_1.*") - .as[A](TypedExpressionEncoder[A]) + val df = + tuple1.dataset.selectExpr("_1.*").as[A](TypedExpressionEncoder[A]) TypedDataset.create(df) case other => @@ -765,217 +985,288 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } } - /** Type-safe projection from type T to Tuple2[A,B] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple2[A,B] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B] - ): TypedDataset[(A, B)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B] + ): TypedDataset[(A, B)] = { implicit val (ea, eb) = (ca.uencoder, cb.uencoder) selectMany(ca, cb) } - /** Type-safe projection from type T to Tuple3[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple3[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C] - ): TypedDataset[(A, B, C)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C] + ): TypedDataset[(A, B, C)] = { implicit val (ea, eb, ec) = (ca.uencoder, cb.uencoder, cc.uencoder) selectMany(ca, cb, cc) } - /** Type-safe projection from type T to Tuple4[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple4[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D] - ): TypedDataset[(A, B, C, D)] = { - implicit val (ea, eb, ec, ed) = (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder) + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D] + ): TypedDataset[(A, B, C, D)] = { + implicit val (ea, eb, ec, ed) = + (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder) selectMany(ca, cb, cc, cd) } - /** Type-safe projection from type T to Tuple5[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple5[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E] - ): TypedDataset[(A, B, C, D, E)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E] + ): TypedDataset[(A, B, C, D, E)] = { implicit val (ea, eb, ec, ed, ee) = (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder) selectMany(ca, cb, cc, cd, ce) } - /** Type-safe projection from type T to Tuple6[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple6[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E, F]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E], - cf: TypedColumn[T, F] - ): TypedDataset[(A, B, C, D, E, F)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E], + cf: TypedColumn[T, F] + ): TypedDataset[(A, B, C, D, E, F)] = { implicit val (ea, eb, ec, ed, ee, ef) = - (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder, cf.uencoder) + ( + ca.uencoder, + cb.uencoder, + cc.uencoder, + cd.uencoder, + ce.uencoder, + cf.uencoder + ) selectMany(ca, cb, cc, cd, ce, cf) } - /** Type-safe projection from type T to Tuple7[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple7[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E, F, G]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E], - cf: TypedColumn[T, F], - cg: TypedColumn[T, G] - ): TypedDataset[(A, B, C, D, E, F, G)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E], + cf: TypedColumn[T, F], + cg: TypedColumn[T, G] + ): TypedDataset[(A, B, C, D, E, F, G)] = { implicit val (ea, eb, ec, ed, ee, ef, eg) = - (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder, cf.uencoder, cg.uencoder) + ( + ca.uencoder, + cb.uencoder, + cc.uencoder, + cd.uencoder, + ce.uencoder, + cf.uencoder, + cg.uencoder + ) selectMany(ca, cb, cc, cd, ce, cf, cg) } - /** Type-safe projection from type T to Tuple8[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple8[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E, F, G, H]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E], - cf: TypedColumn[T, F], - cg: TypedColumn[T, G], - ch: TypedColumn[T, H] - ): TypedDataset[(A, B, C, D, E, F, G, H)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E], + cf: TypedColumn[T, F], + cg: TypedColumn[T, G], + ch: TypedColumn[T, H] + ): TypedDataset[(A, B, C, D, E, F, G, H)] = { implicit val (ea, eb, ec, ed, ee, ef, eg, eh) = - (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder, cf.uencoder, cg.uencoder, ch.uencoder) + ( + ca.uencoder, + cb.uencoder, + cc.uencoder, + cd.uencoder, + ce.uencoder, + cf.uencoder, + cg.uencoder, + ch.uencoder + ) selectMany(ca, cb, cc, cd, ce, cf, cg, ch) } - /** Type-safe projection from type T to Tuple9[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple9[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E, F, G, H, I]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E], - cf: TypedColumn[T, F], - cg: TypedColumn[T, G], - ch: TypedColumn[T, H], - ci: TypedColumn[T, I] - ): TypedDataset[(A, B, C, D, E, F, G, H, I)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E], + cf: TypedColumn[T, F], + cg: TypedColumn[T, G], + ch: TypedColumn[T, H], + ci: TypedColumn[T, I] + ): TypedDataset[(A, B, C, D, E, F, G, H, I)] = { implicit val (ea, eb, ec, ed, ee, ef, eg, eh, ei) = - (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder, cf.uencoder, cg.uencoder, ch.uencoder, ci.uencoder) + ( + ca.uencoder, + cb.uencoder, + cc.uencoder, + cd.uencoder, + ce.uencoder, + cf.uencoder, + cg.uencoder, + ch.uencoder, + ci.uencoder + ) selectMany(ca, cb, cc, cd, ce, cf, cg, ch, ci) } - /** Type-safe projection from type T to Tuple10[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple10[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E, F, G, H, I, J]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E], - cf: TypedColumn[T, F], - cg: TypedColumn[T, G], - ch: TypedColumn[T, H], - ci: TypedColumn[T, I], - cj: TypedColumn[T, J] - ): TypedDataset[(A, B, C, D, E, F, G, H, I, J)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E], + cf: TypedColumn[T, F], + cg: TypedColumn[T, G], + ch: TypedColumn[T, H], + ci: TypedColumn[T, I], + cj: TypedColumn[T, J] + ): TypedDataset[(A, B, C, D, E, F, G, H, I, J)] = { implicit val (ea, eb, ec, ed, ee, ef, eg, eh, ei, ej) = - (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder, cf.uencoder, cg.uencoder, ch.uencoder, ci.uencoder, cj.uencoder) + ( + ca.uencoder, + cb.uencoder, + cc.uencoder, + cd.uencoder, + ce.uencoder, + cf.uencoder, + cg.uencoder, + ch.uencoder, + ci.uencoder, + cj.uencoder + ) selectMany(ca, cb, cc, cd, ce, cf, cg, ch, ci, cj) } object selectMany extends ProductArgs { - def applyProduct[U <: HList, Out0 <: HList, Out](columns: U) - (implicit + + def applyProduct[U <: HList, Out0 <: HList, Out]( + columns: U + )(implicit i0: ColumnTypes.Aux[T, U, Out0], i1: ToTraversable.Aux[U, List, UntypedExpression[T]], i2: Tupler.Aux[Out0, Out], i3: TypedEncoder[Out] ): TypedDataset[Out] = { - val base = dataset.toDF() - .select(columns.toList[UntypedExpression[T]].map(c => new Column(c.expr)):_*) - val selected = base.as[Out](TypedExpressionEncoder[Out]) + val base = dataset + .toDF() + .select( + columns.toList[UntypedExpression[T]].map(c => new Column(c.expr)): _* + ) + val selected = base.as[Out](TypedExpressionEncoder[Out]) - TypedDataset.create[Out](selected) - } + TypedDataset.create[Out](selected) + } } /** Sort each partition in the dataset using the columns selected. */ - def sortWithinPartitions[A: CatalystOrdered](ca: SortedTypedColumn[T, A]): TypedDataset[T] = + def sortWithinPartitions[A: CatalystOrdered]( + ca: SortedTypedColumn[T, A] + ): TypedDataset[T] = sortWithinPartitionsMany(ca) /** Sort each partition in the dataset using the columns selected. */ def sortWithinPartitions[A: CatalystOrdered, B: CatalystOrdered]( - ca: SortedTypedColumn[T, A], - cb: SortedTypedColumn[T, B] - ): TypedDataset[T] = sortWithinPartitionsMany(ca, cb) + ca: SortedTypedColumn[T, A], + cb: SortedTypedColumn[T, B] + ): TypedDataset[T] = sortWithinPartitionsMany(ca, cb) /** Sort each partition in the dataset using the columns selected. */ - def sortWithinPartitions[A: CatalystOrdered, B: CatalystOrdered, C: CatalystOrdered]( - ca: SortedTypedColumn[T, A], - cb: SortedTypedColumn[T, B], - cc: SortedTypedColumn[T, C] - ): TypedDataset[T] = sortWithinPartitionsMany(ca, cb, cc) - - /** Sort each partition in the dataset by the given column expressions - * Default sort order is ascending. - * {{{ - * d.sortWithinPartitionsMany(d('a), d('b).desc, d('c).asc) - * }}} - */ + def sortWithinPartitions[ + A: CatalystOrdered, + B: CatalystOrdered, + C: CatalystOrdered + ](ca: SortedTypedColumn[T, A], + cb: SortedTypedColumn[T, B], + cc: SortedTypedColumn[T, C] + ): TypedDataset[T] = sortWithinPartitionsMany(ca, cb, cc) + + /** + * Sort each partition in the dataset by the given column expressions + * Default sort order is ascending. + * {{{ + * d.sortWithinPartitionsMany(d('a), d('b).desc, d('c).asc) + * }}} + */ object sortWithinPartitionsMany extends ProductArgs { - def applyProduct[U <: HList, O <: HList](columns: U) - (implicit + + def applyProduct[U <: HList, O <: HList]( + columns: U + )(implicit i0: Mapper.Aux[SortedTypedColumn.defaultAscendingPoly.type, U, O], i1: ToTraversable.Aux[O, List, SortedTypedColumn[T, _]] ): TypedDataset[T] = { - val sorted = dataset.toDF() - .sortWithinPartitions(i0(columns).toList[SortedTypedColumn[T, _]].map(_.untyped):_*) + val sorted = dataset + .toDF() + .sortWithinPartitions( + i0(columns).toList[SortedTypedColumn[T, _]].map(_.untyped): _* + ) .as[T](TypedExpressionEncoder[T]) TypedDataset.create[T](sorted) @@ -988,268 +1279,309 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** Orders the TypedDataset using the columns selected. */ def orderBy[A: CatalystOrdered, B: CatalystOrdered]( - ca: SortedTypedColumn[T, A], - cb: SortedTypedColumn[T, B] - ): TypedDataset[T] = orderByMany(ca, cb) - - /** Orders the TypedDataset using the columns selected. */ - def orderBy[A: CatalystOrdered, B: CatalystOrdered, C: CatalystOrdered]( - ca: SortedTypedColumn[T, A], - cb: SortedTypedColumn[T, B], - cc: SortedTypedColumn[T, C] - ): TypedDataset[T] = orderByMany(ca, cb, cc) - - /** Sort the dataset by any number of column expressions. - * Default sort order is ascending. - * {{{ - * d.orderByMany(d('a), d('b).desc, d('c).asc) - * }}} - */ + ca: SortedTypedColumn[T, A], + cb: SortedTypedColumn[T, B] + ): TypedDataset[T] = orderByMany(ca, cb) + + /** Orders the TypedDataset using the columns selected. */ + def orderBy[A: CatalystOrdered, B: CatalystOrdered, C: CatalystOrdered]( + ca: SortedTypedColumn[T, A], + cb: SortedTypedColumn[T, B], + cc: SortedTypedColumn[T, C] + ): TypedDataset[T] = orderByMany(ca, cb, cc) + + /** + * Sort the dataset by any number of column expressions. + * Default sort order is ascending. + * {{{ + * d.orderByMany(d('a), d('b).desc, d('c).asc) + * }}} + */ object orderByMany extends ProductArgs { - def applyProduct[U <: HList, O <: HList](columns: U) - (implicit + + def applyProduct[U <: HList, O <: HList]( + columns: U + )(implicit i0: Mapper.Aux[SortedTypedColumn.defaultAscendingPoly.type, U, O], i1: ToTraversable.Aux[O, List, SortedTypedColumn[T, _]] ): TypedDataset[T] = { - val sorted = dataset.toDF() - .orderBy(i0(columns).toList[SortedTypedColumn[T, _]].map(_.untyped):_*) + val sorted = dataset + .toDF() + .orderBy(i0(columns).toList[SortedTypedColumn[T, _]].map(_.untyped): _*) .as[T](TypedExpressionEncoder[T]) TypedDataset.create[T](sorted) } } - /** Returns a new Dataset as a tuple with the specified - * column dropped. - * Does not allow for dropping from a single column TypedDataset - * - * {{{ - * val d: TypedDataset[Foo(a: String, b: Int...)] = ??? - * val result = TypedDataset[(Int, ...)] = d.drop('a) - * }}} - * @param column column to drop specified as a Symbol - * @param i0 LabelledGeneric derived for T - * @param i1 Remover derived for TRep and column - * @param i2 values of T with column removed - * @param i3 tupler of values - * @param i4 evidence of encoder of the tupled values - * @tparam Out Tupled return type - * @tparam TRep shapeless' record representation of T - * @tparam Removed record of T with column removed - * @tparam ValuesFromRemoved values of T with column removed as an HList - * @tparam V value type of column in T - * @return - */ - def dropTupled[Out, TRep <: HList, Removed <: HList, ValuesFromRemoved <: HList, V] - (column: Witness.Lt[Symbol]) - (implicit + /** + * Returns a new Dataset as a tuple with the specified + * column dropped. + * Does not allow for dropping from a single column TypedDataset + * + * {{{ + * val d: TypedDataset[Foo(a: String, b: Int...)] = ??? + * val result = TypedDataset[(Int, ...)] = d.drop('a) + * }}} + * @param column column to drop specified as a Symbol + * @param i0 LabelledGeneric derived for T + * @param i1 Remover derived for TRep and column + * @param i2 values of T with column removed + * @param i3 tupler of values + * @param i4 evidence of encoder of the tupled values + * @tparam Out Tupled return type + * @tparam TRep shapeless' record representation of T + * @tparam Removed record of T with column removed + * @tparam ValuesFromRemoved values of T with column removed as an HList + * @tparam V value type of column in T + * @return + */ + def dropTupled[ + Out, + TRep <: HList, + Removed <: HList, + ValuesFromRemoved <: HList, + V + ](column: Witness.Lt[Symbol] + )(implicit i0: LabelledGeneric.Aux[T, TRep], i1: Remover.Aux[TRep, column.T, (V, Removed)], i2: Values.Aux[Removed, ValuesFromRemoved], i3: Tupler.Aux[ValuesFromRemoved, Out], i4: TypedEncoder[Out] ): TypedDataset[Out] = { - val dropped = dataset - .toDF() - .drop(column.value.name) - .as[Out](TypedExpressionEncoder[Out]) + val dropped = dataset + .toDF() + .drop(column.value.name) + .as[Out](TypedExpressionEncoder[Out]) - TypedDataset.create[Out](dropped) - } + TypedDataset.create[Out](dropped) + } /** - * Drops columns as necessary to return `U` - * - * @example - * {{{ - * case class X(i: Int, j: Int, k: Boolean) - * case class Y(i: Int, k: Boolean) - * val f: TypedDataset[X] = ??? - * val fNew: TypedDataset[Y] = f.drop[Y] - * }}} - * - * @tparam U the output type - * - * @see [[frameless.TypedDataset#project]] - */ - def drop[U](implicit projector: SmartProject[T,U]): TypedDataset[U] = project[U] - - /** Prepends a new column to the Dataset. - * - * {{{ - * case class X(i: Int, j: Int) - * val f: TypedDataset[X] = TypedDataset.create(X(1,1) :: X(1,1) :: X(1,10) :: Nil) - * val fNew: TypedDataset[(Int,Int,Boolean)] = f.withColumnTupled(f('j) === 10) - * }}} - */ - def withColumnTupled[A: TypedEncoder, H <: HList, FH <: HList, Out] - (ca: TypedColumn[T, A]) - (implicit + * Drops columns as necessary to return `U` + * + * @example + * {{{ + * case class X(i: Int, j: Int, k: Boolean) + * case class Y(i: Int, k: Boolean) + * val f: TypedDataset[X] = ??? + * val fNew: TypedDataset[Y] = f.drop[Y] + * }}} + * + * @tparam U the output type + * + * @see [[frameless.TypedDataset#project]] + */ + def drop[U]( + implicit + projector: SmartProject[T, U] + ): TypedDataset[U] = project[U] + + /** + * Prepends a new column to the Dataset. + * + * {{{ + * case class X(i: Int, j: Int) + * val f: TypedDataset[X] = TypedDataset.create(X(1,1) :: X(1,1) :: X(1,10) :: Nil) + * val fNew: TypedDataset[(Int,Int,Boolean)] = f.withColumnTupled(f('j) === 10) + * }}} + */ + def withColumnTupled[A: TypedEncoder, H <: HList, FH <: HList, Out]( + ca: TypedColumn[T, A] + )(implicit i0: Generic.Aux[T, H], i1: Prepend.Aux[H, A :: HNil, FH], i2: Tupler.Aux[FH, Out], i3: TypedEncoder[Out] ): TypedDataset[Out] = { - // Giving a random name to the new column (the proper name will be given by the Tuple-based encoder) - val selected = dataset.toDF().withColumn("I1X3T9CU1OP0128JYIO76TYZZA3AXHQ18RMI", ca.untyped) - .as[Out](TypedExpressionEncoder[Out]) + // Giving a random name to the new column (the proper name will be given by the Tuple-based encoder) + val selected = dataset + .toDF() + .withColumn("I1X3T9CU1OP0128JYIO76TYZZA3AXHQ18RMI", ca.untyped) + .as[Out](TypedExpressionEncoder[Out]) - TypedDataset.create[Out](selected) + TypedDataset.create[Out](selected) } - /** Returns a new [[frameless.TypedDataset]] with the specified column updated with a new value - * {{{ - * case class X(i: Int, j: Int) - * val f: TypedDataset[X] = TypedDataset.create(X(1,10) :: Nil) - * val fNew: TypedDataset[X] = f.withColumn('j, f('i)) // results in X(1, 1) :: Nil - * }}} - * @param column column given as a symbol to replace - * @param replacement column to replace the value with - * @param i0 Evidence that a column with the correct type and name exists - */ + /** + * Returns a new [[frameless.TypedDataset]] with the specified column updated with a new value + * {{{ + * case class X(i: Int, j: Int) + * val f: TypedDataset[X] = TypedDataset.create(X(1,10) :: Nil) + * val fNew: TypedDataset[X] = f.withColumn('j, f('i)) // results in X(1, 1) :: Nil + * }}} + * @param column column given as a symbol to replace + * @param replacement column to replace the value with + * @param i0 Evidence that a column with the correct type and name exists + */ def withColumnReplaced[A]( - column: Witness.Lt[Symbol], - replacement: TypedColumn[T, A] - )(implicit - i0: TypedColumn.Exists[T, column.T, A] - ): TypedDataset[T] = { - val updated = dataset.toDF().withColumn(column.value.name, replacement.untyped) + column: Witness.Lt[Symbol], + replacement: TypedColumn[T, A] + )(implicit + i0: TypedColumn.Exists[T, column.T, A] + ): TypedDataset[T] = { + val updated = dataset + .toDF() + .withColumn(column.value.name, replacement.untyped) .as[T](TypedExpressionEncoder[T]) TypedDataset.create[T](updated) } - /** Adds a column to a Dataset so long as the specified output type, `U`, has - * an extra column from `T` that has type `A`. - * - * @example - * {{{ - * case class X(i: Int, j: Int) - * case class Y(i: Int, j: Int, k: Boolean) - * val f: TypedDataset[X] = TypedDataset.create(X(1,1) :: X(1,1) :: X(1,10) :: Nil) - * val fNew: TypedDataset[Y] = f.withColumn[Y](f('j) === 10) - * }}} - * @param ca The typed column to add - * @param i0 TypeEncoder for output type U - * @param i1 TypeEncoder for added column type A - * @param i2 the LabelledGeneric derived for T - * @param i3 the LabelledGeneric derived for U - * @param i4 proof no fields have been removed - * @param i5 diff from T to U - * @param i6 keys from newFields - * @param i7 the one and only new key - * @param i8 the one and only new field enforcing the type of A exists - * @param i9 the keys of U - * @param iA allows for traversing the keys of U - * @tparam U the output type - * @tparam A The added column type - * @tparam TRep shapeless' record representation of T - * @tparam URep shapeless' record representation of U - * @tparam UKeys the keys of U as an HList - * @tparam NewFields the added fields to T to get U - * @tparam NewKeys the keys of NewFields as an HList - * @tparam NewKey the first, and only, key in NewKey - * - * @see [[frameless.TypedDataset.WithColumnApply#apply]] - */ + /** + * Adds a column to a Dataset so long as the specified output type, `U`, has + * an extra column from `T` that has type `A`. + * + * @example + * {{{ + * case class X(i: Int, j: Int) + * case class Y(i: Int, j: Int, k: Boolean) + * val f: TypedDataset[X] = TypedDataset.create(X(1,1) :: X(1,1) :: X(1,10) :: Nil) + * val fNew: TypedDataset[Y] = f.withColumn[Y](f('j) === 10) + * }}} + * @param ca The typed column to add + * @param i0 TypeEncoder for output type U + * @param i1 TypeEncoder for added column type A + * @param i2 the LabelledGeneric derived for T + * @param i3 the LabelledGeneric derived for U + * @param i4 proof no fields have been removed + * @param i5 diff from T to U + * @param i6 keys from newFields + * @param i7 the one and only new key + * @param i8 the one and only new field enforcing the type of A exists + * @param i9 the keys of U + * @param iA allows for traversing the keys of U + * @tparam U the output type + * @tparam A The added column type + * @tparam TRep shapeless' record representation of T + * @tparam URep shapeless' record representation of U + * @tparam UKeys the keys of U as an HList + * @tparam NewFields the added fields to T to get U + * @tparam NewKeys the keys of NewFields as an HList + * @tparam NewKey the first, and only, key in NewKey + * + * @see [[frameless.TypedDataset.WithColumnApply#apply]] + */ def withColumn[U] = new WithColumnApply[U] class WithColumnApply[U] { - def apply[A, TRep <: HList, URep <: HList, UKeys <: HList, NewFields <: HList, NewKeys <: HList, NewKey <: Symbol] - (ca: TypedColumn[T, A]) - (implicit - i0: TypedEncoder[U], - i1: TypedEncoder[A], - i2: LabelledGeneric.Aux[T, TRep], - i3: LabelledGeneric.Aux[U, URep], - i4: Diff.Aux[TRep, URep, HNil], - i5: Diff.Aux[URep, TRep, NewFields], - i6: Keys.Aux[NewFields, NewKeys], - i7: IsHCons.Aux[NewKeys, NewKey, HNil], - i8: IsHCons.Aux[NewFields, FieldType[NewKey, A], HNil], - i9: Keys.Aux[URep, UKeys], - iA: ToTraversable.Aux[UKeys, Seq, Symbol] - ): TypedDataset[U] = { + + def apply[ + A, + TRep <: HList, + URep <: HList, + UKeys <: HList, + NewFields <: HList, + NewKeys <: HList, + NewKey <: Symbol + ](ca: TypedColumn[T, A] + )(implicit + i0: TypedEncoder[U], + i1: TypedEncoder[A], + i2: LabelledGeneric.Aux[T, TRep], + i3: LabelledGeneric.Aux[U, URep], + i4: Diff.Aux[TRep, URep, HNil], + i5: Diff.Aux[URep, TRep, NewFields], + i6: Keys.Aux[NewFields, NewKeys], + i7: IsHCons.Aux[NewKeys, NewKey, HNil], + i8: IsHCons.Aux[NewFields, FieldType[NewKey, A], HNil], + i9: Keys.Aux[URep, UKeys], + iA: ToTraversable.Aux[UKeys, Seq, Symbol] + ): TypedDataset[U] = { val newColumnName = i7.head(i6()).name - val dfWithNewColumn = dataset - .toDF() - .withColumn(newColumnName, ca.untyped) + val dfWithNewColumn = dataset.toDF().withColumn(newColumnName, ca.untyped) val newColumns = i9.apply().to[Seq].map(_.name).map(dfWithNewColumn.col) - val selected = dfWithNewColumn - .select(newColumns: _*) - .as[U](TypedExpressionEncoder[U]) + val selected = + dfWithNewColumn.select(newColumns: _*).as[U](TypedExpressionEncoder[U]) TypedDataset.create[U](selected) } } /** - * Explodes a single column at a time. It only compiles if the type of column supports this operation. - * - * @example - * - * {{{ - * case class X(i: Int, j: Array[Int]) - * case class Y(i: Int, j: Int) - * - * val f: TypedDataset[X] = ??? - * val fNew: TypedDataset[Y] = f.explode('j).as[Y] - * }}} - * @param column the column we wish to explode - */ - def explode[A, TRep <: HList, V[_], OutMod <: HList, OutModValues <: HList, Out] - (column: Witness.Lt[Symbol]) - (implicit - i0: TypedColumn.Exists[T, column.T, V[A]], - i1: TypedEncoder[A], - i2: CatalystExplodableCollection[V], - i3: LabelledGeneric.Aux[T, TRep], - i4: Modifier.Aux[TRep, column.T, V[A], A, OutMod], - i5: Values.Aux[OutMod, OutModValues], - i6: Tupler.Aux[OutModValues, Out], - i7: TypedEncoder[Out] - ): TypedDataset[Out] = { - import org.apache.spark.sql.functions.{explode => sparkExplode} + * Explodes a single column at a time. It only compiles if the type of column supports this operation. + * + * @example + * + * {{{ + * case class X(i: Int, j: Array[Int]) + * case class Y(i: Int, j: Int) + * + * val f: TypedDataset[X] = ??? + * val fNew: TypedDataset[Y] = f.explode('j).as[Y] + * }}} + * @param column the column we wish to explode + */ + def explode[ + A, + TRep <: HList, + V[_], + OutMod <: HList, + OutModValues <: HList, + Out + ](column: Witness.Lt[Symbol] + )(implicit + i0: TypedColumn.Exists[T, column.T, V[A]], + i1: TypedEncoder[A], + i2: CatalystExplodableCollection[V], + i3: LabelledGeneric.Aux[T, TRep], + i4: Modifier.Aux[TRep, column.T, V[A], A, OutMod], + i5: Values.Aux[OutMod, OutModValues], + i6: Tupler.Aux[OutModValues, Out], + i7: TypedEncoder[Out] + ): TypedDataset[Out] = { + import org.apache.spark.sql.functions.{ explode => sparkExplode } val df = dataset.toDF() val trans = - df - .withColumn(column.value.name, sparkExplode(df(column.value.name))) + df.withColumn(column.value.name, sparkExplode(df(column.value.name))) .as[Out](TypedExpressionEncoder[Out]) TypedDataset.create[Out](trans) } /** - * Explodes a single column at a time. It only compiles if the type of column supports this operation. - * - * @example - * - * {{{ - * case class X(i: Int, j: Map[Int, Int]) - * case class Y(i: Int, j: (Int, Int)) - * - * val f: TypedDataset[X] = ??? - * val fNew: TypedDataset[Y] = f.explodeMap('j).as[Y] - * }}} - * @param column the column we wish to explode - */ - def explodeMap[A, B, V[_, _], TRep <: HList, OutMod <: HList, OutModValues <: HList, Out] - (column: Witness.Lt[Symbol]) - (implicit - i0: TypedColumn.Exists[T, column.T, V[A, B]], - i1: TypedEncoder[A], - i2: TypedEncoder[B], - i3: LabelledGeneric.Aux[T, TRep], - i4: Modifier.Aux[TRep, column.T, V[A,B], (A, B), OutMod], - i5: Values.Aux[OutMod, OutModValues], - i6: Tupler.Aux[OutModValues, Out], - i7: TypedEncoder[Out] - ): TypedDataset[Out] = { - import org.apache.spark.sql.functions.{explode => sparkExplode, struct => sparkStruct, col => sparkCol} + * Explodes a single column at a time. It only compiles if the type of column supports this operation. + * + * @example + * + * {{{ + * case class X(i: Int, j: Map[Int, Int]) + * case class Y(i: Int, j: (Int, Int)) + * + * val f: TypedDataset[X] = ??? + * val fNew: TypedDataset[Y] = f.explodeMap('j).as[Y] + * }}} + * @param column the column we wish to explode + */ + def explodeMap[ + A, + B, + V[_, _], + TRep <: HList, + OutMod <: HList, + OutModValues <: HList, + Out + ](column: Witness.Lt[Symbol] + )(implicit + i0: TypedColumn.Exists[T, column.T, V[A, B]], + i1: TypedEncoder[A], + i2: TypedEncoder[B], + i3: LabelledGeneric.Aux[T, TRep], + i4: Modifier.Aux[TRep, column.T, V[A, B], (A, B), OutMod], + i5: Values.Aux[OutMod, OutModValues], + i6: Tupler.Aux[OutModValues, Out], + i7: TypedEncoder[Out] + ): TypedDataset[Out] = { + import org.apache.spark.sql.functions.{ + explode => sparkExplode, + struct => sparkStruct, + col => sparkCol + } val df = dataset.toDF() // select all columns, all original columns and [key, value] columns appeared after the map explode @@ -1271,7 +1603,10 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val exploded // map explode explodes it into [key, value] columns // the only way to put it into a column is to create a struct - .withColumn(columnRenamed, sparkStruct(exploded("key"), exploded("value"))) + .withColumn( + columnRenamed, + sparkStruct(exploded("key"), exploded("value")) + ) // selecting only original columns, we don't need [key, value] columns left in the DataFrame after the map explode .select(columns: _*) // rename columns back and form the result @@ -1281,72 +1616,81 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } /** - * Flattens a column of type Option[A]. Compiles only if the selected column is of type Option[A]. - * - * - * @example - * - * {{{ - * case class X(i: Int, j: Option[Int]) - * case class Y(i: Int, j: Int) - * - * val f: TypedDataset[X] = ??? - * val fNew: TypedDataset[Y] = f.flattenOption('j).as[Y] - * }}} - * - * @param column the column we wish to flatten - */ - def flattenOption[A, TRep <: HList, V[_], OutMod <: HList, OutModValues <: HList, Out] - (column: Witness.Lt[Symbol]) - (implicit - i0: TypedColumn.Exists[T, column.T, V[A]], - i1: TypedEncoder[A], - i2: V[A] =:= Option[A], - i3: LabelledGeneric.Aux[T, TRep], - i4: Modifier.Aux[TRep, column.T, V[A], A, OutMod], - i5: Values.Aux[OutMod, OutModValues], - i6: Tupler.Aux[OutModValues, Out], - i7: TypedEncoder[Out] - ): TypedDataset[Out] = { + * Flattens a column of type Option[A]. Compiles only if the selected column is of type Option[A]. + * + * @example + * + * {{{ + * case class X(i: Int, j: Option[Int]) + * case class Y(i: Int, j: Int) + * + * val f: TypedDataset[X] = ??? + * val fNew: TypedDataset[Y] = f.flattenOption('j).as[Y] + * }}} + * + * @param column the column we wish to flatten + */ + def flattenOption[ + A, + TRep <: HList, + V[_], + OutMod <: HList, + OutModValues <: HList, + Out + ](column: Witness.Lt[Symbol] + )(implicit + i0: TypedColumn.Exists[T, column.T, V[A]], + i1: TypedEncoder[A], + i2: V[A] =:= Option[A], + i3: LabelledGeneric.Aux[T, TRep], + i4: Modifier.Aux[TRep, column.T, V[A], A, OutMod], + i5: Values.Aux[OutMod, OutModValues], + i6: Tupler.Aux[OutModValues, Out], + i7: TypedEncoder[Out] + ): TypedDataset[Out] = { val df = dataset.toDF() - val trans = df.filter(df(column.value.name).isNotNull). - as[Out](TypedExpressionEncoder[Out]) + val trans = df + .filter(df(column.value.name).isNotNull) + .as[Out](TypedExpressionEncoder[Out]) TypedDataset.create[Out](trans) } } object TypedDataset { - def create[A](data: Seq[A]) - (implicit + + def create[A]( + data: Seq[A] + )(implicit encoder: TypedEncoder[A], sqlContext: SparkSession ): TypedDataset[A] = { - val dataset = sqlContext.createDataset(data)(TypedExpressionEncoder[A]) + val dataset = sqlContext.createDataset(data)(TypedExpressionEncoder[A]) - TypedDataset.create[A](dataset) - } + TypedDataset.create[A](dataset) + } - def create[A](data: RDD[A]) - (implicit + def create[A]( + data: RDD[A] + )(implicit encoder: TypedEncoder[A], sqlContext: SparkSession ): TypedDataset[A] = { - val dataset = sqlContext.createDataset(data)(TypedExpressionEncoder[A]) + val dataset = sqlContext.createDataset(data)(TypedExpressionEncoder[A]) - TypedDataset.create[A](dataset) - } + TypedDataset.create[A](dataset) + } def create[A: TypedEncoder](dataset: Dataset[A]): TypedDataset[A] = createUnsafe(dataset.toDF()) /** - * Creates a [[frameless.TypedDataset]] from a Spark [[org.apache.spark.sql.DataFrame]]. - * Note that the names and types need to align! - * - * This is an unsafe operation: If the schemas do not align, - * the error will be captured at runtime (not during compilation). - */ + * Creates a [[frameless.TypedDataset]] from a Spark [[org.apache.spark.sql.DataFrame]]. + * Note that the names and types need to align! + * + * This is an unsafe operation: If the schemas do not align, + * the error will be captured at runtime (not during compilation). + */ def createUnsafe[A: TypedEncoder](df: DataFrame): TypedDataset[A] = { val e = TypedEncoder[A] val output: Seq[Attribute] = df.queryExecution.analyzed.output @@ -1358,7 +1702,8 @@ object TypedDataset { throw new IllegalStateException( s"Unsupported creation of TypedDataset with ${targetFields.size} column(s) " + s"from a DataFrame with ${output.size} columns. " + - "Try to `select()` the proper columns in the right order before calling `create()`.") + "Try to `select()` the proper columns in the right order before calling `create()`." + ) } // Adapt names if they are not the same (note: types still might not match) @@ -1368,7 +1713,7 @@ object TypedDataset { val canSelect = targetColNames.toSet.subsetOf(output.map(_.name).toSet) val reshaped = if (shouldReshape && canSelect) { - df.select(targetColNames.head, targetColNames.tail:_*) + df.select(targetColNames.head, targetColNames.tail: _*) } else if (shouldReshape) { df.toDF(targetColNames: _*) } else { @@ -1378,9 +1723,14 @@ object TypedDataset { new TypedDataset[A](reshaped.as[A](TypedExpressionEncoder[A])) } - /** Prefer `TypedDataset.create` over `TypedDataset.unsafeCreate` unless you - * know what you are doing. */ - @deprecated("Prefer TypedDataset.create over TypedDataset.unsafeCreate", "0.3.0") + /** + * Prefer `TypedDataset.create` over `TypedDataset.unsafeCreate` unless you + * know what you are doing. + */ + @deprecated( + "Prefer TypedDataset.create over TypedDataset.unsafeCreate", + "0.3.0" + ) def unsafeCreate[A: TypedEncoder](dataset: Dataset[A]): TypedDataset[A] = { new TypedDataset[A](dataset) } diff --git a/dataset/src/main/scala/frameless/TypedEncoder.scala b/dataset/src/main/scala/frameless/TypedEncoder.scala index 3a10f9781..b72ff7712 100644 --- a/dataset/src/main/scala/frameless/TypedEncoder.scala +++ b/dataset/src/main/scala/frameless/TypedEncoder.scala @@ -6,7 +6,11 @@ import java.time.{ Duration, Instant, LocalDate, Period } import java.sql.Timestamp import scala.reflect.ClassTag import FramelessInternals.UserDefinedType -import org.apache.spark.sql.catalyst.expressions.{ Expression, UnsafeArrayData, Literal } +import org.apache.spark.sql.catalyst.expressions.{ + Expression, + UnsafeArrayData, + Literal +} import org.apache.spark.sql.catalyst.util.{ ArrayBasedMapData, DateTimeUtils, @@ -23,7 +27,7 @@ import com.sparkutils.shim.expressions.{ MapObjects5 => MapObjects, ExternalMapToCatalyst7 => ExternalMapToCatalyst } -import frameless.{reflection => ScalaReflection} +import frameless.{ reflection => ScalaReflection } import org.apache.spark.sql.shim.{ StaticInvoke4 => StaticInvoke, NewInstance4 => NewInstance, diff --git a/dataset/src/main/scala/frameless/functions/NonAggregateFunctions.scala b/dataset/src/main/scala/frameless/functions/NonAggregateFunctions.scala index 45fdcb1a2..396b7ff43 100644 --- a/dataset/src/main/scala/frameless/functions/NonAggregateFunctions.scala +++ b/dataset/src/main/scala/frameless/functions/NonAggregateFunctions.scala @@ -1,534 +1,718 @@ package frameless package functions -import org.apache.spark.sql.{Column, functions => sparkFunctions} +import org.apache.spark.sql.{ Column, functions => sparkFunctions } -import com.sparkutils.shim.expressions.{functions => shimFunctions} +import com.sparkutils.shim.expressions.{ functions => shimFunctions } import scala.util.matching.Regex trait NonAggregateFunctions { - /** Non-Aggregate function: calculates the SHA-2 digest of a binary column and returns the value as a 40 character hex string - * - * apache/spark - */ - def sha2[T](column: AbstractTypedColumn[T, Array[Byte]], numBits: Int): column.ThisType[T, String] = + + /** + * Non-Aggregate function: calculates the SHA-2 digest of a binary column and returns the value as a 40 character hex string + * + * apache/spark + */ + def sha2[T]( + column: AbstractTypedColumn[T, Array[Byte]], + numBits: Int + ): column.ThisType[T, String] = column.typed(sparkFunctions.sha2(column.untyped, numBits)) - /** Non-Aggregate function: calculates the SHA-1 digest of a binary column and returns the value as a 40 character hex string - * - * apache/spark - */ + /** + * Non-Aggregate function: calculates the SHA-1 digest of a binary column and returns the value as a 40 character hex string + * + * apache/spark + */ def sha1[T](column: AbstractTypedColumn[T, Array[Byte]]): column.ThisType[T, String] = column.typed(sparkFunctions.sha1(column.untyped)) - /** Non-Aggregate function: returns a cyclic redundancy check value of a binary column as long. - * - * apache/spark - */ + /** + * Non-Aggregate function: returns a cyclic redundancy check value of a binary column as long. + * + * apache/spark + */ def crc32[T](column: AbstractTypedColumn[T, Array[Byte]]): column.ThisType[T, Long] = column.typed(sparkFunctions.crc32(column.untyped)) + /** - * Non-Aggregate function: returns the negated value of column. - * - * apache/spark - */ - def negate[A, B, T](column: AbstractTypedColumn[T,A])( - implicit i0: CatalystNumericWithJavaBigDecimal[A, B], - i1: TypedEncoder[B] - ): column.ThisType[T,B] = + * Non-Aggregate function: returns the negated value of column. + * + * apache/spark + */ + def negate[A, B, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystNumericWithJavaBigDecimal[A, B], + i1: TypedEncoder[B] + ): column.ThisType[T, B] = column.typed(sparkFunctions.negate(column.untyped)) /** - * Non-Aggregate function: logical not. - * - * apache/spark - */ - def not[T](column: AbstractTypedColumn[T,Boolean]): column.ThisType[T,Boolean] = + * Non-Aggregate function: logical not. + * + * apache/spark + */ + def not[T](column: AbstractTypedColumn[T, Boolean]): column.ThisType[T, Boolean] = column.typed(sparkFunctions.not(column.untyped)) /** - * Non-Aggregate function: Convert a number in a string column from one base to another. - * - * apache/spark - */ - def conv[T](column: AbstractTypedColumn[T,String], fromBase: Int, toBase: Int): column.ThisType[T,String] = - column.typed(sparkFunctions.conv(column.untyped,fromBase,toBase)) + * Non-Aggregate function: Convert a number in a string column from one base to another. + * + * apache/spark + */ + def conv[T]( + column: AbstractTypedColumn[T, String], + fromBase: Int, + toBase: Int + ): column.ThisType[T, String] = + column.typed(sparkFunctions.conv(column.untyped, fromBase, toBase)) - /** Non-Aggregate function: Converts an angle measured in radians to an approximately equivalent angle measured in degrees. - * - * apache/spark - */ - def degrees[A,T](column: AbstractTypedColumn[T,A]): column.ThisType[T,Double] = + /** + * Non-Aggregate function: Converts an angle measured in radians to an approximately equivalent angle measured in degrees. + * + * apache/spark + */ + def degrees[A, T](column: AbstractTypedColumn[T, A]): column.ThisType[T, Double] = column.typed(sparkFunctions.degrees(column.untyped)) - /** Non-Aggregate function: returns the ceiling of a numeric column - * - * apache/spark - */ - def ceil[A, B, T](column: AbstractTypedColumn[T, A]) - (implicit + /** + * Non-Aggregate function: returns the ceiling of a numeric column + * + * apache/spark + */ + def ceil[A, B, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystRound[A, B], + i1: TypedEncoder[B] + ): column.ThisType[T, B] = + column.typed(sparkFunctions.ceil(column.untyped))(i1) + + /** + * Non-Aggregate function: returns the floor of a numeric column + * + * apache/spark + */ + def floor[A, B, T]( + column: AbstractTypedColumn[T, A] + )(implicit i0: CatalystRound[A, B], i1: TypedEncoder[B] ): column.ThisType[T, B] = - column.typed(sparkFunctions.ceil(column.untyped))(i1) - - /** Non-Aggregate function: returns the floor of a numeric column - * - * apache/spark - */ - def floor[A, B, T](column: AbstractTypedColumn[T, A]) - (implicit - i0: CatalystRound[A, B], - i1: TypedEncoder[B] - ): column.ThisType[T, B] = column.typed(sparkFunctions.floor(column.untyped))(i1) - /** Non-Aggregate function: unsigned shift the the given value numBits right. If given long, will return long else it will return an integer. - * - * apache/spark - */ - def shiftRightUnsigned[A, B, T](column: AbstractTypedColumn[T, A], numBits: Int) - (implicit + /** + * Non-Aggregate function: unsigned shift the the given value numBits right. If given long, will return long else it will return an integer. + * + * apache/spark + */ + def shiftRightUnsigned[A, B, T]( + column: AbstractTypedColumn[T, A], + numBits: Int + )(implicit i0: CatalystBitShift[A, B], i1: TypedEncoder[B] ): column.ThisType[T, B] = - column.typed(shimFunctions.shiftRightUnsigned(column.untyped, numBits)) - - /** Non-Aggregate function: shift the the given value numBits right. If given long, will return long else it will return an integer. - * - * apache/spark - */ - def shiftRight[A, B, T](column: AbstractTypedColumn[T, A], numBits: Int) - (implicit + column.typed(shimFunctions.shiftRightUnsigned(column.untyped, numBits)) + + /** + * Non-Aggregate function: shift the the given value numBits right. If given long, will return long else it will return an integer. + * + * apache/spark + */ + def shiftRight[A, B, T]( + column: AbstractTypedColumn[T, A], + numBits: Int + )(implicit i0: CatalystBitShift[A, B], i1: TypedEncoder[B] ): column.ThisType[T, B] = - column.typed(shimFunctions.shiftRight(column.untyped, numBits)) - - /** Non-Aggregate function: shift the the given value numBits left. If given long, will return long else it will return an integer. - * - * apache/spark - */ - def shiftLeft[A, B, T](column: AbstractTypedColumn[T, A], numBits: Int) - (implicit + column.typed(shimFunctions.shiftRight(column.untyped, numBits)) + + /** + * Non-Aggregate function: shift the the given value numBits left. If given long, will return long else it will return an integer. + * + * apache/spark + */ + def shiftLeft[A, B, T]( + column: AbstractTypedColumn[T, A], + numBits: Int + )(implicit i0: CatalystBitShift[A, B], i1: TypedEncoder[B] ): column.ThisType[T, B] = column.typed(shimFunctions.shiftLeft(column.untyped, numBits)) - - /** Non-Aggregate function: returns the absolute value of a numeric column - * - * apache/spark - */ - def abs[A, B, T](column: AbstractTypedColumn[T, A]) - (implicit - i0: CatalystNumericWithJavaBigDecimal[A, B], - i1: TypedEncoder[B] + + /** + * Non-Aggregate function: returns the absolute value of a numeric column + * + * apache/spark + */ + def abs[A, B, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystNumericWithJavaBigDecimal[A, B], + i1: TypedEncoder[B] ): column.ThisType[T, B] = - column.typed(sparkFunctions.abs(column.untyped))(i1) - - /** Non-Aggregate function: Computes the cosine of the given value. - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def cos[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = - column.typed(sparkFunctions.cos(column.cast[Double].untyped)) - - /** Non-Aggregate function: Computes the hyperbolic cosine of the given value. - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def cosh[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = - column.typed(sparkFunctions.cosh(column.cast[Double].untyped)) - - /** Non-Aggregate function: Computes the signum of the given value. - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def signum[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = + column.typed(sparkFunctions.abs(column.untyped))(i1) + + /** + * Non-Aggregate function: Computes the cosine of the given value. + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def cos[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = + column.typed(sparkFunctions.cos(column.cast[Double].untyped)) + + /** + * Non-Aggregate function: Computes the hyperbolic cosine of the given value. + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def cosh[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = + column.typed(sparkFunctions.cosh(column.cast[Double].untyped)) + + /** + * Non-Aggregate function: Computes the signum of the given value. + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def signum[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.signum(column.cast[Double].untyped)) - /** Non-Aggregate function: Computes the sine of the given value. - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def sin[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = - column.typed(sparkFunctions.sin(column.cast[Double].untyped)) - - /** Non-Aggregate function: Computes the hyperbolic sine of the given value. - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def sinh[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = - column.typed(sparkFunctions.sinh(column.cast[Double].untyped)) - - /** Non-Aggregate function: Computes the tangent of the given column. - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def tan[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = - column.typed(sparkFunctions.tan(column.cast[Double].untyped)) - - /** Non-Aggregate function: Computes the hyperbolic tangent of the given value. - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def tanh[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = - column.typed(sparkFunctions.tanh(column.cast[Double].untyped)) - - /** Non-Aggregate function: returns the acos of a numeric column - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def acos[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = - column.typed(sparkFunctions.acos(column.cast[Double].untyped)) - - /** Non-Aggregate function: returns true if value is contained with in the array in the specified column - * - * apache/spark - */ - def arrayContains[C[_]: CatalystCollection, A, T](column: AbstractTypedColumn[T, C[A]], value: A): column.ThisType[T, Boolean] = + /** + * Non-Aggregate function: Computes the sine of the given value. + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def sin[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = + column.typed(sparkFunctions.sin(column.cast[Double].untyped)) + + /** + * Non-Aggregate function: Computes the hyperbolic sine of the given value. + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def sinh[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = + column.typed(sparkFunctions.sinh(column.cast[Double].untyped)) + + /** + * Non-Aggregate function: Computes the tangent of the given column. + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def tan[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = + column.typed(sparkFunctions.tan(column.cast[Double].untyped)) + + /** + * Non-Aggregate function: Computes the hyperbolic tangent of the given value. + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def tanh[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = + column.typed(sparkFunctions.tanh(column.cast[Double].untyped)) + + /** + * Non-Aggregate function: returns the acos of a numeric column + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def acos[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = + column.typed(sparkFunctions.acos(column.cast[Double].untyped)) + + /** + * Non-Aggregate function: returns true if value is contained with in the array in the specified column + * + * apache/spark + */ + def arrayContains[C[_]: CatalystCollection, A, T]( + column: AbstractTypedColumn[T, C[A]], + value: A + ): column.ThisType[T, Boolean] = column.typed(sparkFunctions.array_contains(column.untyped, value)) - /** Non-Aggregate function: returns the atan of a numeric column - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def atan[A, T](column: AbstractTypedColumn[T,A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = - column.typed(sparkFunctions.atan(column.cast[Double].untyped)) - - /** Non-Aggregate function: returns the asin of a numeric column - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def asin[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = - column.typed(sparkFunctions.asin(column.cast[Double].untyped)) - - /** Non-Aggregate function: returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def atan2[A, B, T](l: TypedColumn[T, A], r: TypedColumn[T, B]) - (implicit + /** + * Non-Aggregate function: returns the atan of a numeric column + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def atan[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = + column.typed(sparkFunctions.atan(column.cast[Double].untyped)) + + /** + * Non-Aggregate function: returns the asin of a numeric column + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def asin[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = + column.typed(sparkFunctions.asin(column.cast[Double].untyped)) + + /** + * Non-Aggregate function: returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def atan2[A, B, T]( + l: TypedColumn[T, A], + r: TypedColumn[T, B] + )(implicit i0: CatalystCast[A, Double], i1: CatalystCast[B, Double] ): TypedColumn[T, Double] = - r.typed(sparkFunctions.atan2(l.cast[Double].untyped, r.cast[Double].untyped)) - - /** Non-Aggregate function: returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). - * - * Spark will expect a Double value for this expression. See: - * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] - * apache/spark - */ - def atan2[A, B, T](l: TypedAggregate[T, A], r: TypedAggregate[T, B]) - (implicit + r.typed( + sparkFunctions.atan2(l.cast[Double].untyped, r.cast[Double].untyped) + ) + + /** + * Non-Aggregate function: returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + * + * Spark will expect a Double value for this expression. See: + * [[https://github.com/apache/spark/blob/4a3c09601ba69f7d49d1946bb6f20f5cfe453031/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala#L67]] + * apache/spark + */ + def atan2[A, B, T]( + l: TypedAggregate[T, A], + r: TypedAggregate[T, B] + )(implicit i0: CatalystCast[A, Double], i1: CatalystCast[B, Double] ): TypedAggregate[T, Double] = - r.typed(sparkFunctions.atan2(l.cast[Double].untyped, r.cast[Double].untyped)) - - def atan2[B, T](l: Double, r: TypedColumn[T, B]) - (implicit i0: CatalystCast[B, Double]): TypedColumn[T, Double] = - atan2(r.lit(l), r) - - def atan2[A, T](l: TypedColumn[T, A], r: Double) - (implicit i0: CatalystCast[A, Double]): TypedColumn[T, Double] = - atan2(l, l.lit(r)) - - def atan2[B, T](l: Double, r: TypedAggregate[T, B]) - (implicit i0: CatalystCast[B, Double]): TypedAggregate[T, Double] = - atan2(r.lit(l), r) - - def atan2[A, T](l: TypedAggregate[T, A], r: Double) - (implicit i0: CatalystCast[A, Double]): TypedAggregate[T, Double] = - atan2(l, l.lit(r)) - - /** Non-Aggregate function: returns the square root value of a numeric column. - * - * apache/spark - */ - def sqrt[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = + r.typed( + sparkFunctions.atan2(l.cast[Double].untyped, r.cast[Double].untyped) + ) + + def atan2[B, T]( + l: Double, + r: TypedColumn[T, B] + )(implicit + i0: CatalystCast[B, Double] + ): TypedColumn[T, Double] = + atan2(r.lit(l), r) + + def atan2[A, T]( + l: TypedColumn[T, A], + r: Double + )(implicit + i0: CatalystCast[A, Double] + ): TypedColumn[T, Double] = + atan2(l, l.lit(r)) + + def atan2[B, T]( + l: Double, + r: TypedAggregate[T, B] + )(implicit + i0: CatalystCast[B, Double] + ): TypedAggregate[T, Double] = + atan2(r.lit(l), r) + + def atan2[A, T]( + l: TypedAggregate[T, A], + r: Double + )(implicit + i0: CatalystCast[A, Double] + ): TypedAggregate[T, Double] = + atan2(l, l.lit(r)) + + /** + * Non-Aggregate function: returns the square root value of a numeric column. + * + * apache/spark + */ + def sqrt[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.sqrt(column.cast[Double].untyped)) - /** Non-Aggregate function: returns the cubic root value of a numeric column. - * - * apache/spark - */ - def cbrt[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = + /** + * Non-Aggregate function: returns the cubic root value of a numeric column. + * + * apache/spark + */ + def cbrt[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.cbrt(column.cast[Double].untyped)) - /** Non-Aggregate function: returns the exponential value of a numeric column. - * - * apache/spark - */ - def exp[A, T](column: AbstractTypedColumn[T, A]) - (implicit i0: CatalystCast[A, Double]): column.ThisType[T, Double] = + /** + * Non-Aggregate function: returns the exponential value of a numeric column. + * + * apache/spark + */ + def exp[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.exp(column.cast[Double].untyped)) - /** Non-Aggregate function: Returns the value of the column `e` rounded to 0 decimal places with HALF_UP round mode. - * - * apache/spark - */ - def round[A, B, T](column: AbstractTypedColumn[T, A])( - implicit i0: CatalystNumericWithJavaBigDecimal[A, B], i1: TypedEncoder[B] - ): column.ThisType[T, B] = + /** + * Non-Aggregate function: Returns the value of the column `e` rounded to 0 decimal places with HALF_UP round mode. + * + * apache/spark + */ + def round[A, B, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystNumericWithJavaBigDecimal[A, B], + i1: TypedEncoder[B] + ): column.ThisType[T, B] = column.typed(sparkFunctions.round(column.untyped))(i1) - /** Non-Aggregate function: Round the value of `e` to `scale` decimal places with HALF_UP round mode - * if `scale` is greater than or equal to 0 or at integral part when `scale` is less than 0. - * - * apache/spark - */ - def round[A, B, T](column: AbstractTypedColumn[T, A], scale: Int)( - implicit i0: CatalystNumericWithJavaBigDecimal[A, B], i1: TypedEncoder[B] - ): column.ThisType[T, B] = + /** + * Non-Aggregate function: Round the value of `e` to `scale` decimal places with HALF_UP round mode + * if `scale` is greater than or equal to 0 or at integral part when `scale` is less than 0. + * + * apache/spark + */ + def round[A, B, T]( + column: AbstractTypedColumn[T, A], + scale: Int + )(implicit + i0: CatalystNumericWithJavaBigDecimal[A, B], + i1: TypedEncoder[B] + ): column.ThisType[T, B] = column.typed(sparkFunctions.round(column.untyped, scale))(i1) - /** Non-Aggregate function: Bankers Rounding - returns the rounded to 0 decimal places value with HALF_EVEN round mode - * of a numeric column. - * - * apache/spark - */ - def bround[A, B, T](column: AbstractTypedColumn[T, A])( - implicit i0: CatalystNumericWithJavaBigDecimal[A, B], i1: TypedEncoder[B] - ): column.ThisType[T, B] = + /** + * Non-Aggregate function: Bankers Rounding - returns the rounded to 0 decimal places value with HALF_EVEN round mode + * of a numeric column. + * + * apache/spark + */ + def bround[A, B, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystNumericWithJavaBigDecimal[A, B], + i1: TypedEncoder[B] + ): column.ThisType[T, B] = column.typed(sparkFunctions.bround(column.untyped))(i1) - /** Non-Aggregate function: Bankers Rounding - returns the rounded to `scale` decimal places value with HALF_EVEN round mode - * of a numeric column. If `scale` is greater than or equal to 0 or at integral part when `scale` is less than 0. - * - * apache/spark - */ - def bround[A, B, T](column: AbstractTypedColumn[T, A], scale: Int)( - implicit i0: CatalystNumericWithJavaBigDecimal[A, B], i1: TypedEncoder[B] - ): column.ThisType[T, B] = + /** + * Non-Aggregate function: Bankers Rounding - returns the rounded to `scale` decimal places value with HALF_EVEN round mode + * of a numeric column. If `scale` is greater than or equal to 0 or at integral part when `scale` is less than 0. + * + * apache/spark + */ + def bround[A, B, T]( + column: AbstractTypedColumn[T, A], + scale: Int + )(implicit + i0: CatalystNumericWithJavaBigDecimal[A, B], + i1: TypedEncoder[B] + ): column.ThisType[T, B] = column.typed(sparkFunctions.bround(column.untyped, scale))(i1) /** - * Computes the natural logarithm of the given value. - * - * apache/spark - */ - def log[A, T](column: AbstractTypedColumn[T, A])( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Computes the natural logarithm of the given value. + * + * apache/spark + */ + def log[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.log(column.untyped)) /** - * Returns the first argument-base logarithm of the second argument. - * - * apache/spark - */ - def log[A, T](base: Double, column: AbstractTypedColumn[T, A])( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Returns the first argument-base logarithm of the second argument. + * + * apache/spark + */ + def log[A, T]( + base: Double, + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.log(base, column.untyped)) /** - * Computes the logarithm of the given column in base 2. - * - * apache/spark - */ - def log2[A, T](column: AbstractTypedColumn[T, A])( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Computes the logarithm of the given column in base 2. + * + * apache/spark + */ + def log2[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.log2(column.untyped)) /** - * Computes the natural logarithm of the given value plus one. - * - * apache/spark - */ - def log1p[A, T](column: AbstractTypedColumn[T, A])( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Computes the natural logarithm of the given value plus one. + * + * apache/spark + */ + def log1p[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.log1p(column.untyped)) /** - * Computes the logarithm of the given column in base 10. - * - * apache/spark - */ - def log10[A, T](column: AbstractTypedColumn[T, A])( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Computes the logarithm of the given column in base 10. + * + * apache/spark + */ + def log10[A, T]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.log10(column.untyped)) - /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - * - * apache/spark - */ - def hypot[A, T](column: AbstractTypedColumn[T, A], column2: AbstractTypedColumn[T, A])( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * apache/spark + */ + def hypot[A, T]( + column: AbstractTypedColumn[T, A], + column2: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.hypot(column.untyped, column2.untyped)) /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - * - * apache/spark - */ - def hypot[A, T](column: AbstractTypedColumn[T, A], l: Double)( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * apache/spark + */ + def hypot[A, T]( + column: AbstractTypedColumn[T, A], + l: Double + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.hypot(column.untyped, l)) /** - * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. - * - * apache/spark - */ - def hypot[A, T](l: Double, column: AbstractTypedColumn[T, A])( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + * + * apache/spark + */ + def hypot[A, T]( + l: Double, + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.hypot(l, column.untyped)) /** - * Returns the value of the first argument raised to the power of the second argument. - * - * apache/spark - */ - def pow[A, T](column: AbstractTypedColumn[T, A], column2: AbstractTypedColumn[T, A])( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Returns the value of the first argument raised to the power of the second argument. + * + * apache/spark + */ + def pow[A, T]( + column: AbstractTypedColumn[T, A], + column2: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.pow(column.untyped, column2.untyped)) /** - * Returns the value of the first argument raised to the power of the second argument. - * - * apache/spark - */ - def pow[A, T](column: AbstractTypedColumn[T, A], l: Double)( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Returns the value of the first argument raised to the power of the second argument. + * + * apache/spark + */ + def pow[A, T]( + column: AbstractTypedColumn[T, A], + l: Double + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.pow(column.untyped, l)) /** - * Returns the value of the first argument raised to the power of the second argument. - * - * apache/spark - */ - def pow[A, T](l: Double, column: AbstractTypedColumn[T, A])( - implicit i0: CatalystCast[A, Double] - ): column.ThisType[T, Double] = + * Returns the value of the first argument raised to the power of the second argument. + * + * apache/spark + */ + def pow[A, T]( + l: Double, + column: AbstractTypedColumn[T, A] + )(implicit + i0: CatalystCast[A, Double] + ): column.ThisType[T, Double] = column.typed(sparkFunctions.pow(l, column.untyped)) /** - * Returns the positive value of dividend mod divisor. - * - * apache/spark - */ - def pmod[A, T](column: AbstractTypedColumn[T, A], column2: AbstractTypedColumn[T, A])( - implicit i0: TypedEncoder[A] - ): column.ThisType[T, A] = + * Returns the positive value of dividend mod divisor. + * + * apache/spark + */ + def pmod[A, T]( + column: AbstractTypedColumn[T, A], + column2: AbstractTypedColumn[T, A] + )(implicit + i0: TypedEncoder[A] + ): column.ThisType[T, A] = column.typed(sparkFunctions.pmod(column.untyped, column2.untyped)) - - /** Non-Aggregate function: Returns the string representation of the binary value of the given long - * column. For example, bin("12") returns "1100". - * - * apache/spark - */ + /** + * Non-Aggregate function: Returns the string representation of the binary value of the given long + * column. For example, bin("12") returns "1100". + * + * apache/spark + */ def bin[T](column: AbstractTypedColumn[T, Long]): column.ThisType[T, String] = column.typed(sparkFunctions.bin(column.untyped)) /** - * Calculates the MD5 digest of a binary column and returns the value - * as a 32 character hex string. - * - * apache/spark - */ - def md5[T, A](column: AbstractTypedColumn[T, A])(implicit i0: TypedEncoder[A]): column.ThisType[T, String] = + * Calculates the MD5 digest of a binary column and returns the value + * as a 32 character hex string. + * + * apache/spark + */ + def md5[T, A]( + column: AbstractTypedColumn[T, A] + )(implicit + i0: TypedEncoder[A] + ): column.ThisType[T, String] = column.typed(sparkFunctions.md5(column.untyped)) /** - * Computes the factorial of the given value. - * - * apache/spark - */ - def factorial[T](column: AbstractTypedColumn[T, Long])(implicit i0: TypedEncoder[Long]): column.ThisType[T, Long] = + * Computes the factorial of the given value. + * + * apache/spark + */ + def factorial[T]( + column: AbstractTypedColumn[T, Long] + )(implicit + i0: TypedEncoder[Long] + ): column.ThisType[T, Long] = column.typed(sparkFunctions.factorial(column.untyped)) - /** Non-Aggregate function: Computes bitwise NOT. - * - * apache/spark - */ - def bitwiseNOT[A: CatalystBitwise, T](column: AbstractTypedColumn[T, A]): column.ThisType[T, A] = + /** + * Non-Aggregate function: Computes bitwise NOT. + * + * apache/spark + */ + def bitwiseNOT[A: CatalystBitwise, T]( + column: AbstractTypedColumn[T, A] + ): column.ThisType[T, A] = column.typed(shimFunctions.bitwiseNOT(column.untyped))(column.uencoder) - /** Non-Aggregate function: file name of the current Spark task. Empty string if row did not originate from - * a file - * - * apache/spark - */ + /** + * Non-Aggregate function: file name of the current Spark task. Empty string if row did not originate from + * a file + * + * apache/spark + */ def inputFileName[T](): TypedColumn[T, String] = new TypedColumn[T, String](sparkFunctions.input_file_name()) - /** Non-Aggregate function: generates monotonically increasing id - * - * apache/spark - */ + /** + * Non-Aggregate function: generates monotonically increasing id + * + * apache/spark + */ def monotonicallyIncreasingId[T](): TypedColumn[T, Long] = { new TypedColumn[T, Long](sparkFunctions.monotonically_increasing_id()) } - /** Non-Aggregate function: Evaluates a list of conditions and returns one of multiple - * possible result expressions. If none match, otherwise is returned - * {{{ - * when(ds('boolField), ds('a)) - * .when(ds('otherBoolField), lit(123)) - * .otherwise(ds('b)) - * }}} - * apache/spark - */ - def when[T, A](condition: AbstractTypedColumn[T, Boolean], value: AbstractTypedColumn[T, A]): When[T, A] = + /** + * Non-Aggregate function: Evaluates a list of conditions and returns one of multiple + * possible result expressions. If none match, otherwise is returned + * {{{ + * when(ds('boolField), ds('a)) + * .when(ds('otherBoolField), lit(123)) + * .otherwise(ds('b)) + * }}} + * apache/spark + */ + def when[T, A]( + condition: AbstractTypedColumn[T, Boolean], + value: AbstractTypedColumn[T, A] + ): When[T, A] = new When[T, A](condition, value) class When[T, A] private (untypedC: Column) { - private[functions] def this(condition: AbstractTypedColumn[T, Boolean], value: AbstractTypedColumn[T, A]) = + private[functions] def this( + condition: AbstractTypedColumn[T, Boolean], + value: AbstractTypedColumn[T, A] + ) = this(sparkFunctions.when(condition.untyped, value.untyped)) - def when(condition: AbstractTypedColumn[T, Boolean], value: AbstractTypedColumn[T, A]): When[T, A] = + def when( + condition: AbstractTypedColumn[T, Boolean], + value: AbstractTypedColumn[T, A] + ): When[T, A] = new When[T, A](untypedC.when(condition.untyped, value.untyped)) def otherwise(value: AbstractTypedColumn[T, A]): value.ThisType[T, A] = @@ -539,172 +723,219 @@ trait NonAggregateFunctions { // String functions ////////////////////////////////////////////////////////////////////////////////////////////// - - /** Non-Aggregate function: takes the first letter of a string column and returns the ascii int value in a new column - * - * apache/spark - */ + /** + * Non-Aggregate function: takes the first letter of a string column and returns the ascii int value in a new column + * + * apache/spark + */ def ascii[T](column: AbstractTypedColumn[T, String]): column.ThisType[T, Int] = column.typed(sparkFunctions.ascii(column.untyped)) - /** Non-Aggregate function: Computes the BASE64 encoding of a binary column and returns it as a string column. - * This is the reverse of unbase64. - * - * apache/spark - */ + /** + * Non-Aggregate function: Computes the BASE64 encoding of a binary column and returns it as a string column. + * This is the reverse of unbase64. + * + * apache/spark + */ def base64[T](column: AbstractTypedColumn[T, Array[Byte]]): column.ThisType[T, String] = column.typed(sparkFunctions.base64(column.untyped)) - /** Non-Aggregate function: Decodes a BASE64 encoded string column and returns it as a binary column. - * This is the reverse of base64. - * - * apache/spark - */ + /** + * Non-Aggregate function: Decodes a BASE64 encoded string column and returns it as a binary column. + * This is the reverse of base64. + * + * apache/spark + */ def unbase64[T](column: AbstractTypedColumn[T, String]): column.ThisType[T, Array[Byte]] = column.typed(sparkFunctions.unbase64(column.untyped)) - /** Non-Aggregate function: Concatenates multiple input string columns together into a single string column. - * @note varargs make it harder to generalize so we overload the method for [[TypedColumn]] and [[TypedAggregate]] - * - * apache/spark - */ + /** + * Non-Aggregate function: Concatenates multiple input string columns together into a single string column. + * @note varargs make it harder to generalize so we overload the method for [[TypedColumn]] and [[TypedAggregate]] + * + * apache/spark + */ def concat[T](columns: TypedColumn[T, String]*): TypedColumn[T, String] = new TypedColumn(sparkFunctions.concat(columns.map(_.untyped): _*)) - /** Non-Aggregate function: Concatenates multiple input string columns together into a single string column. - * @note varargs make it harder to generalize so we overload the method for [[TypedColumn]] and [[TypedAggregate]] - * - * apache/spark - */ + /** + * Non-Aggregate function: Concatenates multiple input string columns together into a single string column. + * @note varargs make it harder to generalize so we overload the method for [[TypedColumn]] and [[TypedAggregate]] + * + * apache/spark + */ def concat[T](columns: TypedAggregate[T, String]*): TypedAggregate[T, String] = new TypedAggregate(sparkFunctions.concat(columns.map(_.untyped): _*)) - /** Non-Aggregate function: Concatenates multiple input string columns together into a single string column, - * using the given separator. - * @note varargs make it harder to generalize so we overload the method for [[TypedColumn]] and [[TypedAggregate]] - * - * apache/spark - */ - def concatWs[T](sep: String, columns: TypedAggregate[T, String]*): TypedAggregate[T, String] = - new TypedAggregate(sparkFunctions.concat_ws(sep, columns.map(_.untyped): _*)) - - /** Non-Aggregate function: Concatenates multiple input string columns together into a single string column, - * using the given separator. - * @note varargs make it harder to generalize so we overload the method for [[TypedColumn]] and [[TypedAggregate]] - * - * apache/spark - */ + /** + * Non-Aggregate function: Concatenates multiple input string columns together into a single string column, + * using the given separator. + * @note varargs make it harder to generalize so we overload the method for [[TypedColumn]] and [[TypedAggregate]] + * + * apache/spark + */ + def concatWs[T]( + sep: String, + columns: TypedAggregate[T, String]* + ): TypedAggregate[T, String] = + new TypedAggregate( + sparkFunctions.concat_ws(sep, columns.map(_.untyped): _*) + ) + + /** + * Non-Aggregate function: Concatenates multiple input string columns together into a single string column, + * using the given separator. + * @note varargs make it harder to generalize so we overload the method for [[TypedColumn]] and [[TypedAggregate]] + * + * apache/spark + */ def concatWs[T](sep: String, columns: TypedColumn[T, String]*): TypedColumn[T, String] = new TypedColumn(sparkFunctions.concat_ws(sep, columns.map(_.untyped): _*)) - /** Non-Aggregate function: Locates the position of the first occurrence of substring column - * in given string - * - * @note The position is not zero based, but 1 based index. Returns 0 if substr - * could not be found in str. - * - * apache/spark - */ - def instr[T](str: AbstractTypedColumn[T, String], substring: String): str.ThisType[T, Int] = + /** + * Non-Aggregate function: Locates the position of the first occurrence of substring column + * in given string + * + * @note The position is not zero based, but 1 based index. Returns 0 if substr + * could not be found in str. + * + * apache/spark + */ + def instr[T]( + str: AbstractTypedColumn[T, String], + substring: String + ): str.ThisType[T, Int] = str.typed(sparkFunctions.instr(str.untyped, substring)) - /** Non-Aggregate function: Computes the length of a given string. - * - * apache/spark - */ - //TODO: Also for binary + /** + * Non-Aggregate function: Computes the length of a given string. + * + * apache/spark + */ + // TODO: Also for binary def length[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Int] = str.typed(sparkFunctions.length(str.untyped)) - /** Non-Aggregate function: Computes the Levenshtein distance of the two given string columns. - * - * apache/spark - */ - def levenshtein[T](l: TypedColumn[T, String], r: TypedColumn[T, String]): TypedColumn[T, Int] = + /** + * Non-Aggregate function: Computes the Levenshtein distance of the two given string columns. + * + * apache/spark + */ + def levenshtein[T]( + l: TypedColumn[T, String], + r: TypedColumn[T, String] + ): TypedColumn[T, Int] = l.typed(sparkFunctions.levenshtein(l.untyped, r.untyped)) - /** Non-Aggregate function: Computes the Levenshtein distance of the two given string columns. - * - * apache/spark - */ - def levenshtein[T](l: TypedAggregate[T, String], r: TypedAggregate[T, String]): TypedAggregate[T, Int] = + /** + * Non-Aggregate function: Computes the Levenshtein distance of the two given string columns. + * + * apache/spark + */ + def levenshtein[T]( + l: TypedAggregate[T, String], + r: TypedAggregate[T, String] + ): TypedAggregate[T, Int] = l.typed(sparkFunctions.levenshtein(l.untyped, r.untyped)) - /** Non-Aggregate function: Converts a string column to lower case. - * - * apache/spark - */ + /** + * Non-Aggregate function: Converts a string column to lower case. + * + * apache/spark + */ def lower[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, String] = str.typed(sparkFunctions.lower(str.untyped)) - /** Non-Aggregate function: Left-pad the string column with pad to a length of len. If the string column is longer - * than len, the return value is shortened to len characters. - * - * apache/spark - */ - def lpad[T](str: AbstractTypedColumn[T, String], - len: Int, - pad: String): str.ThisType[T, String] = + /** + * Non-Aggregate function: Left-pad the string column with pad to a length of len. If the string column is longer + * than len, the return value is shortened to len characters. + * + * apache/spark + */ + def lpad[T]( + str: AbstractTypedColumn[T, String], + len: Int, + pad: String + ): str.ThisType[T, String] = str.typed(sparkFunctions.lpad(str.untyped, len, pad)) - /** Non-Aggregate function: Trim the spaces from left end for the specified string value. - * - * apache/spark - */ + /** + * Non-Aggregate function: Trim the spaces from left end for the specified string value. + * + * apache/spark + */ def ltrim[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, String] = str.typed(sparkFunctions.ltrim(str.untyped)) - /** Non-Aggregate function: Replace all substrings of the specified string value that match regexp with rep. - * - * apache/spark - */ - def regexpReplace[T](str: AbstractTypedColumn[T, String], - pattern: Regex, - replacement: String): str.ThisType[T, String] = - str.typed(sparkFunctions.regexp_replace(str.untyped, pattern.regex, replacement)) - + /** + * Non-Aggregate function: Replace all substrings of the specified string value that match regexp with rep. + * + * apache/spark + */ + def regexpReplace[T]( + str: AbstractTypedColumn[T, String], + pattern: Regex, + replacement: String + ): str.ThisType[T, String] = + str.typed( + sparkFunctions.regexp_replace(str.untyped, pattern.regex, replacement) + ) - /** Non-Aggregate function: Reverses the string column and returns it as a new string column. - * - * apache/spark - */ + /** + * Non-Aggregate function: Reverses the string column and returns it as a new string column. + * + * apache/spark + */ def reverse[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, String] = str.typed(sparkFunctions.reverse(str.untyped)) - /** Non-Aggregate function: Right-pad the string column with pad to a length of len. - * If the string column is longer than len, the return value is shortened to len characters. - * - * apache/spark - */ - def rpad[T](str: AbstractTypedColumn[T, String], len: Int, pad: String): str.ThisType[T, String] = + /** + * Non-Aggregate function: Right-pad the string column with pad to a length of len. + * If the string column is longer than len, the return value is shortened to len characters. + * + * apache/spark + */ + def rpad[T]( + str: AbstractTypedColumn[T, String], + len: Int, + pad: String + ): str.ThisType[T, String] = str.typed(sparkFunctions.rpad(str.untyped, len, pad)) - /** Non-Aggregate function: Trim the spaces from right end for the specified string value. - * - * apache/spark - */ + /** + * Non-Aggregate function: Trim the spaces from right end for the specified string value. + * + * apache/spark + */ def rtrim[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, String] = str.typed(sparkFunctions.rtrim(str.untyped)) - /** Non-Aggregate function: Substring starts at `pos` and is of length `len` - * - * apache/spark - */ - //TODO: Also for byte array - def substring[T](str: AbstractTypedColumn[T, String], pos: Int, len: Int): str.ThisType[T, String] = + /** + * Non-Aggregate function: Substring starts at `pos` and is of length `len` + * + * apache/spark + */ + // TODO: Also for byte array + def substring[T]( + str: AbstractTypedColumn[T, String], + pos: Int, + len: Int + ): str.ThisType[T, String] = str.typed(sparkFunctions.substring(str.untyped, pos, len)) - /** Non-Aggregate function: Trim the spaces from both ends for the specified string column. - * - * apache/spark - */ + /** + * Non-Aggregate function: Trim the spaces from both ends for the specified string column. + * + * apache/spark + */ def trim[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, String] = str.typed(sparkFunctions.trim(str.untyped)) - /** Non-Aggregate function: Converts a string column to upper case. - * - * apache/spark - */ + /** + * Non-Aggregate function: Converts a string column to upper case. + * + * apache/spark + */ def upper[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, String] = str.typed(sparkFunctions.upper(str.untyped)) @@ -712,93 +943,103 @@ trait NonAggregateFunctions { // DateTime functions ////////////////////////////////////////////////////////////////////////////////////////////// - /** Non-Aggregate function: Extracts the year as an integer from a given date/timestamp/string. - * - * Differs from `Column#year` by wrapping it's result into an `Option`. - * - * apache/spark - */ + /** + * Non-Aggregate function: Extracts the year as an integer from a given date/timestamp/string. + * + * Differs from `Column#year` by wrapping it's result into an `Option`. + * + * apache/spark + */ def year[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Option[Int]] = str.typed(sparkFunctions.year(str.untyped)) - /** Non-Aggregate function: Extracts the quarter as an integer from a given date/timestamp/string. - * - * Differs from `Column#quarter` by wrapping it's result into an `Option`. - * - * apache/spark - */ + /** + * Non-Aggregate function: Extracts the quarter as an integer from a given date/timestamp/string. + * + * Differs from `Column#quarter` by wrapping it's result into an `Option`. + * + * apache/spark + */ def quarter[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Option[Int]] = str.typed(sparkFunctions.quarter(str.untyped)) - /** Non-Aggregate function Extracts the month as an integer from a given date/timestamp/string. - * - * Differs from `Column#month` by wrapping it's result into an `Option`. - * - * apache/spark - */ + /** + * Non-Aggregate function Extracts the month as an integer from a given date/timestamp/string. + * + * Differs from `Column#month` by wrapping it's result into an `Option`. + * + * apache/spark + */ def month[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Option[Int]] = str.typed(sparkFunctions.month(str.untyped)) - /** Non-Aggregate function: Extracts the day of the week as an integer from a given date/timestamp/string. - * - * Differs from `Column#dayofweek` by wrapping it's result into an `Option`. - * - * apache/spark - */ + /** + * Non-Aggregate function: Extracts the day of the week as an integer from a given date/timestamp/string. + * + * Differs from `Column#dayofweek` by wrapping it's result into an `Option`. + * + * apache/spark + */ def dayofweek[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Option[Int]] = str.typed(sparkFunctions.dayofweek(str.untyped)) - /** Non-Aggregate function: Extracts the day of the month as an integer from a given date/timestamp/string. - * - * Differs from `Column#dayofmonth` by wrapping it's result into an `Option`. - * - * apache/spark - */ + /** + * Non-Aggregate function: Extracts the day of the month as an integer from a given date/timestamp/string. + * + * Differs from `Column#dayofmonth` by wrapping it's result into an `Option`. + * + * apache/spark + */ def dayofmonth[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Option[Int]] = str.typed(sparkFunctions.dayofmonth(str.untyped)) - /** Non-Aggregate function: Extracts the day of the year as an integer from a given date/timestamp/string. - * - * Differs from `Column#dayofyear` by wrapping it's result into an `Option`. - * - * apache/spark - */ + /** + * Non-Aggregate function: Extracts the day of the year as an integer from a given date/timestamp/string. + * + * Differs from `Column#dayofyear` by wrapping it's result into an `Option`. + * + * apache/spark + */ def dayofyear[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Option[Int]] = str.typed(sparkFunctions.dayofyear(str.untyped)) - /** Non-Aggregate function: Extracts the hours as an integer from a given date/timestamp/string. - * - * Differs from `Column#hour` by wrapping it's result into an `Option`. - * - * apache/spark - */ + /** + * Non-Aggregate function: Extracts the hours as an integer from a given date/timestamp/string. + * + * Differs from `Column#hour` by wrapping it's result into an `Option`. + * + * apache/spark + */ def hour[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Option[Int]] = str.typed(sparkFunctions.hour(str.untyped)) - /** Non-Aggregate function: Extracts the minutes as an integer from a given date/timestamp/string. - * - * Differs from `Column#minute` by wrapping it's result into an `Option`. - * - * apache/spark - */ + /** + * Non-Aggregate function: Extracts the minutes as an integer from a given date/timestamp/string. + * + * Differs from `Column#minute` by wrapping it's result into an `Option`. + * + * apache/spark + */ def minute[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Option[Int]] = str.typed(sparkFunctions.minute(str.untyped)) - /** Non-Aggregate function: Extracts the seconds as an integer from a given date/timestamp/string. - * - * Differs from `Column#second` by wrapping it's result into an `Option`. - * - * apache/spark - */ + /** + * Non-Aggregate function: Extracts the seconds as an integer from a given date/timestamp/string. + * + * Differs from `Column#second` by wrapping it's result into an `Option`. + * + * apache/spark + */ def second[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Option[Int]] = str.typed(sparkFunctions.second(str.untyped)) - /** Non-Aggregate function: Extracts the week number as an integer from a given date/timestamp/string. - * - * Differs from `Column#weekofyear` by wrapping it's result into an `Option`. - * - * apache/spark - */ + /** + * Non-Aggregate function: Extracts the week number as an integer from a given date/timestamp/string. + * + * Differs from `Column#weekofyear` by wrapping it's result into an `Option`. + * + * apache/spark + */ def weekofyear[T](str: AbstractTypedColumn[T, String]): str.ThisType[T, Option[Int]] = str.typed(sparkFunctions.weekofyear(str.untyped)) } diff --git a/dataset/src/main/scala/frameless/functions/package.scala b/dataset/src/main/scala/frameless/functions/package.scala index 391852dce..543925e00 100644 --- a/dataset/src/main/scala/frameless/functions/package.scala +++ b/dataset/src/main/scala/frameless/functions/package.scala @@ -1,6 +1,6 @@ package frameless -import frameless.{reflection => ScalaReflection} +import frameless.{ reflection => ScalaReflection } import scala.reflect.ClassTag import shapeless._ diff --git a/dataset/src/main/scala/frameless/ops/GroupByOps.scala b/dataset/src/main/scala/frameless/ops/GroupByOps.scala index 1fbb314e5..d63870c9b 100644 --- a/dataset/src/main/scala/frameless/ops/GroupByOps.scala +++ b/dataset/src/main/scala/frameless/ops/GroupByOps.scala @@ -3,11 +3,7 @@ package ops import org.apache.spark.sql.catalyst.analysis.UnresolvedAlias import org.apache.spark.sql.catalyst.plans.logical.Project -import org.apache.spark.sql.{ - Column, - Dataset, - RelationalGroupedDataset -} +import org.apache.spark.sql.{ Column, Dataset, RelationalGroupedDataset } import shapeless._ import shapeless.ops.hlist.{ Length, diff --git a/dataset/src/main/scala/frameless/reflection/package.scala b/dataset/src/main/scala/frameless/reflection/package.scala index aa4551225..5a38baa71 100644 --- a/dataset/src/main/scala/frameless/reflection/package.scala +++ b/dataset/src/main/scala/frameless/reflection/package.scala @@ -71,7 +71,9 @@ package object reflection { // But, for other types, we do need to erasure it. For example, we need to erasure // `scala.Any` to `java.lang.Object` in order to load it from Java ClassLoader. // Please see SPARK-17368 & SPARK-31190 for more details. - if (isSubtype(tpe, localTypeOf[AnyVal]) && !tpe.toString.startsWith("scala")) { + if ( + isSubtype(tpe, localTypeOf[AnyVal]) && !tpe.toString.startsWith("scala") + ) { tpe } else { tpe.erasure diff --git a/dataset/src/test/scala/frameless/UdtEncodedClass.scala b/dataset/src/test/scala/frameless/UdtEncodedClass.scala index 1c000c58c..b98f74a11 100644 --- a/dataset/src/test/scala/frameless/UdtEncodedClass.scala +++ b/dataset/src/test/scala/frameless/UdtEncodedClass.scala @@ -1,14 +1,19 @@ package frameless import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} +import org.apache.spark.sql.catalyst.expressions.{ + GenericInternalRow, + UnsafeArrayData +} import org.apache.spark.sql.types._ import FramelessInternals.UserDefinedType @SQLUserDefinedType(udt = classOf[UdtEncodedClassUdt]) class UdtEncodedClass(val a: Int, val b: Array[Double]) { + override def equals(other: Any): Boolean = other match { - case that: UdtEncodedClass => a == that.a && java.util.Arrays.equals(b, that.b) + case that: UdtEncodedClass => + a == that.a && java.util.Arrays.equals(b, that.b) case _ => false } @@ -25,11 +30,18 @@ object UdtEncodedClass { } class UdtEncodedClassUdt extends UserDefinedType[UdtEncodedClass] { + def sqlType: DataType = { - StructType(Seq( - StructField("a", IntegerType, nullable = false), - StructField("b", ArrayType(DoubleType, containsNull = false), nullable = false) - )) + StructType( + Seq( + StructField("a", IntegerType, nullable = false), + StructField( + "b", + ArrayType(DoubleType, containsNull = false), + nullable = false + ) + ) + ) } def serialize(obj: UdtEncodedClass): InternalRow = { @@ -40,7 +52,8 @@ class UdtEncodedClassUdt extends UserDefinedType[UdtEncodedClass] { } def deserialize(datum: Any): UdtEncodedClass = datum match { - case row: InternalRow => new UdtEncodedClass(row.getInt(0), row.getArray(1).toDoubleArray()) + case row: InternalRow => + new UdtEncodedClass(row.getInt(0), row.getArray(1).toDoubleArray()) } def userClass: Class[UdtEncodedClass] = classOf[UdtEncodedClass] diff --git a/ml/src/main/scala/frameless/ml/package.scala b/ml/src/main/scala/frameless/ml/package.scala index 1ce56980b..32e4b2184 100644 --- a/ml/src/main/scala/frameless/ml/package.scala +++ b/ml/src/main/scala/frameless/ml/package.scala @@ -1,13 +1,15 @@ package frameless import FramelessInternals.UserDefinedType -import org.apache.spark.ml.{FramelessInternals => MLFramelessInternals} -import org.apache.spark.ml.linalg.{Matrix, Vector} +import org.apache.spark.sql.shim.{ utils => MLFramelessInternals } +import org.apache.spark.ml.linalg.{ Matrix, Vector } package object ml { - implicit val mlVectorUdt: UserDefinedType[Vector] = MLFramelessInternals.vectorUdt + implicit val mlVectorUdt: UserDefinedType[Vector] = + MLFramelessInternals.vectorUdt - implicit val mlMatrixUdt: UserDefinedType[Matrix] = MLFramelessInternals.matrixUdt + implicit val mlMatrixUdt: UserDefinedType[Matrix] = + MLFramelessInternals.matrixUdt } diff --git a/ml/src/main/scala/org/apache/spark/ml/FramelessInternals.scala b/ml/src/main/scala/org/apache/spark/ml/FramelessInternals.scala deleted file mode 100644 index bec43cd11..000000000 --- a/ml/src/main/scala/org/apache/spark/ml/FramelessInternals.scala +++ /dev/null @@ -1,13 +0,0 @@ -package org.apache.spark.ml - -import org.apache.spark.ml.linalg.{MatrixUDT, VectorUDT} - -object FramelessInternals { - - // because org.apache.spark.ml.linalg.VectorUDT is private[spark] - val vectorUdt = new VectorUDT - - // because org.apache.spark.ml.linalg.MatrixUDT is private[spark] - val matrixUdt = new MatrixUDT - -}