From eb66711c00ee526e49c60459290ffb708c7296ca Mon Sep 17 00:00:00 2001 From: Olivier Blanvillain Date: Fri, 4 Aug 2017 11:04:57 +0200 Subject: [PATCH 1/6] Remove TypedColumn's first type parameter This change reduces the type saftely of column expressions. For instance, it's now possible to typecheck some nonsensical `select` as follows: ``` ds1.select(ds2('a)) // This use to be a type error! ``` This simplification is a required step to support joins, which is also source compatible. Safety can hopefuly be recovered by having `select` take a `TypedColumn[T] => TypedColumn[C]` function and removing the apply method on TypedDataset. --- build.sbt | 1 + .../scala/frameless/FramelessSyntax.scala | 2 +- .../main/scala/frameless/TypedColumn.scala | 73 ++++--- .../main/scala/frameless/TypedDataset.scala | 197 +++++++++--------- .../functions/AggregateFunctions.scala | 114 +++++----- .../main/scala/frameless/functions/Udf.scala | 30 +-- .../frameless/functions/UnaryFunctions.scala | 18 +- .../scala/frameless/functions/package.scala | 2 +- .../src/main/scala/frameless/implicits.scala | 40 ++-- .../scala/frameless/ops/AggregateTypes.scala | 4 +- .../scala/frameless/ops/ColumnTypes.scala | 16 +- .../main/scala/frameless/ops/GroupByOps.scala | 34 +-- .../src/test/scala/frameless/ColTests.scala | 8 +- .../test/scala/frameless/SelectTests.scala | 2 +- .../src/test/scala/frameless/WidenTests.scala | 4 +- .../functions/AggregateFunctionsTests.scala | 20 +- .../functions/UnaryFunctionsTest.scala | 2 +- .../scala/frameless/ops/ColumnTypesTest.scala | 4 +- 18 files changed, 286 insertions(+), 285 deletions(-) diff --git a/build.sbt b/build.sbt index 9abc5d38e..7f61a0d85 100644 --- a/build.sbt +++ b/build.sbt @@ -68,6 +68,7 @@ lazy val commonScalacOptions = Seq( "-feature", "-unchecked", "-Xfatal-warnings", + // "-Xlog-implicits", "-Xlint:-missing-interpolator,_", "-Yinline-warnings", "-Yno-adapted-args", diff --git a/dataset/src/main/scala/frameless/FramelessSyntax.scala b/dataset/src/main/scala/frameless/FramelessSyntax.scala index fa102200e..a3d38a76f 100644 --- a/dataset/src/main/scala/frameless/FramelessSyntax.scala +++ b/dataset/src/main/scala/frameless/FramelessSyntax.scala @@ -4,7 +4,7 @@ import org.apache.spark.sql.{Column, DataFrame, Dataset} trait FramelessSyntax { implicit class ColumnSyntax(self: Column) { - def typed[T, U: TypedEncoder]: TypedColumn[T, U] = new TypedColumn[T, U](self) + def typed[T, U: TypedEncoder]: TypedColumn[U] = new TypedColumn[U](self) } implicit class DatasetSyntax[T: TypedEncoder](self: Dataset[T]) { diff --git a/dataset/src/main/scala/frameless/TypedColumn.scala b/dataset/src/main/scala/frameless/TypedColumn.scala index 61a1b1114..9c0009e44 100644 --- a/dataset/src/main/scala/frameless/TypedColumn.scala +++ b/dataset/src/main/scala/frameless/TypedColumn.scala @@ -10,7 +10,7 @@ import shapeless._ import scala.annotation.implicitNotFound -sealed trait UntypedExpression[T] { +sealed trait UntypedExpression { def expr: Expression def uencoder: TypedEncoder[_] override def toString: String = expr.toString() @@ -22,14 +22,13 @@ sealed trait UntypedExpression[T] { * at https://github.com/apache/spark, licensed under Apache v2.0 available at * http://www.apache.org/licenses/LICENSE-2.0 * - * @tparam T type of dataset * @tparam U type of column */ -sealed class TypedColumn[T, U]( +sealed class TypedColumn[U]( val expr: Expression)( implicit val uencoder: TypedEncoder[U] -) extends UntypedExpression[T] { self => +) extends UntypedExpression { self => /** From an untyped Column to a [[TypedColumn]] * @@ -46,7 +45,7 @@ sealed class TypedColumn[T, U]( private def withExpr(newExpr: Expression): Column = new Column(newExpr) - private def equalsTo(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = withExpr { + private def equalsTo(other: TypedColumn[U]): TypedColumn[Boolean] = withExpr { if (uencoder.nullable && uencoder.targetDataType.typeName != "struct") EqualNullSafe(self.expr, other.expr) else EqualTo(self.expr, other.expr) }.typed @@ -58,7 +57,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def ===(other: U): TypedColumn[T, Boolean] = equalsTo(lit(other)) + def ===(other: U): TypedColumn[Boolean] = equalsTo(lit(other)) /** Equality test. * {{{ @@ -67,7 +66,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def ===(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = equalsTo(other) + def ===(other: TypedColumn[U]): TypedColumn[Boolean] = equalsTo(other) /** Inequality test. * {{{ @@ -76,7 +75,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def =!=(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = withExpr { + def =!=(other: TypedColumn[U]): TypedColumn[Boolean] = withExpr { Not(equalsTo(other).expr) }.typed @@ -87,7 +86,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def =!=(other: U): TypedColumn[T, Boolean] = withExpr { + def =!=(other: U): TypedColumn[Boolean] = withExpr { Not(equalsTo(lit(other)).expr) }.typed @@ -95,14 +94,14 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def isNone(implicit isOption: U <:< Option[_]): TypedColumn[T, Boolean] = - equalsTo(lit[U,T](None.asInstanceOf[U])) + def isNone(implicit isOption: U <:< Option[_]): TypedColumn[Boolean] = + equalsTo(lit(None.asInstanceOf[U])) /** True if the current expression is an Option and it's not None. * * apache/spark */ - def isNotNone(implicit isOption: U <:< Option[_]): TypedColumn[T, Boolean] = withExpr { + def isNotNone(implicit isOption: U <:< Option[_]): TypedColumn[Boolean] = withExpr { Not(equalsTo(lit(None.asInstanceOf[U])).expr) }.typed @@ -114,7 +113,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def plus(other: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = + def plus(other: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = self.untyped.plus(other.untyped).typed /** Sum of this expression and another expression. @@ -125,7 +124,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def +(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = plus(u) + def +(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = plus(u) /** Sum of this expression (column) with a constant. * {{{ @@ -136,7 +135,7 @@ sealed class TypedColumn[T, U]( * @param u a constant of the same type * apache/spark */ - def +(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = self.untyped.plus(u).typed + def +(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[U] = self.untyped.plus(u).typed /** Unary minus, i.e. negate the expression. * {{{ @@ -146,7 +145,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def unary_-(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = (-self.untyped).typed + def unary_-(implicit n: CatalystNumeric[U]): TypedColumn[U] = (-self.untyped).typed /** Subtraction. Subtract the other expression from this expression. * {{{ @@ -156,7 +155,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def minus(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = + def minus(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = self.untyped.minus(u.untyped).typed /** Subtraction. Subtract the other expression from this expression. @@ -167,7 +166,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def -(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = minus(u) + def -(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = minus(u) /** Subtraction. Subtract the other expression from this expression. * {{{ @@ -178,7 +177,7 @@ sealed class TypedColumn[T, U]( * @param u a constant of the same type * apache/spark */ - def -(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = self.untyped.minus(u).typed + def -(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[U] = self.untyped.minus(u).typed /** Multiplication of this expression and another expression. * {{{ @@ -188,7 +187,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def multiply(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = + def multiply(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = self.untyped.multiply(u.untyped).typed /** Multiplication of this expression and another expression. @@ -199,7 +198,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def *(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = multiply(u) + def *(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = multiply(u) /** Multiplication of this expression a constant. * {{{ @@ -209,7 +208,7 @@ sealed class TypedColumn[T, U]( * * apache/spark */ - def *(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = self.untyped.multiply(u).typed + def *(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[U] = self.untyped.multiply(u).typed /** * Division this expression by another expression. @@ -221,7 +220,7 @@ sealed class TypedColumn[T, U]( * @param u another column of the same type * apache/spark */ - def divide(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, Double] = + def divide(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[Double] = self.untyped.divide(u.untyped).typed /** @@ -234,7 +233,7 @@ sealed class TypedColumn[T, U]( * @param u another column of the same type * apache/spark */ - def /(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, Double] = divide(u) + def /(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[Double] = divide(u) /** * Division this expression by another expression. @@ -246,14 +245,14 @@ sealed class TypedColumn[T, U]( * @param u a constant of the same type * apache/spark */ - def /(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[T, Double] = self.untyped.divide(u).typed + def /(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[Double] = self.untyped.divide(u).typed /** Casts the column to a different type. * {{{ * df.select(df('a).cast[Int]) * }}} */ - def cast[A: TypedEncoder](implicit c: CatalystCast[U, A]): TypedColumn[T, A] = + def cast[A: TypedEncoder](implicit c: CatalystCast[U, A]): TypedColumn[A] = self.untyped.cast(TypedEncoder[A].targetDataType).typed } @@ -262,10 +261,10 @@ sealed class TypedColumn[T, U]( * @tparam T type of dataset * @tparam U type of column for `groupBy` */ -sealed class TypedAggregate[T, U](val expr: Expression)( +sealed class TypedAggregate[U](val expr: Expression)( implicit val uencoder: TypedEncoder[U] -) extends UntypedExpression[T] { +) extends UntypedExpression { def this(column: Column)(implicit uenc: TypedEncoder[U]) { this(FramelessInternals.expr(column)) @@ -308,15 +307,15 @@ object TypedColumn { ): Exists[T, K, V] = new Exists[T, K, V] {} } - implicit class OrderedTypedColumnSyntax[T, U: CatalystOrdered](col: TypedColumn[T, U]) { - def <(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = (col.untyped < other.untyped).typed - def <=(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = (col.untyped <= other.untyped).typed - def >(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = (col.untyped > other.untyped).typed - def >=(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = (col.untyped >= other.untyped).typed + implicit class OrderedTypedColumnSyntax[U: CatalystOrdered](col: TypedColumn[U]) { + def <(other: TypedColumn[U]): TypedColumn[Boolean] = (col.untyped < other.untyped).typed + def <=(other: TypedColumn[U]): TypedColumn[Boolean] = (col.untyped <= other.untyped).typed + def >(other: TypedColumn[U]): TypedColumn[Boolean] = (col.untyped > other.untyped).typed + def >=(other: TypedColumn[U]): TypedColumn[Boolean] = (col.untyped >= other.untyped).typed - def <(other: U): TypedColumn[T, Boolean] = (col.untyped < lit(other)(col.uencoder).untyped).typed - def <=(other: U): TypedColumn[T, Boolean] = (col.untyped <= lit(other)(col.uencoder).untyped).typed - def >(other: U): TypedColumn[T, Boolean] = (col.untyped > lit(other)(col.uencoder).untyped).typed - def >=(other: U): TypedColumn[T, Boolean] = (col.untyped >= lit(other)(col.uencoder).untyped).typed + def <(other: U): TypedColumn[Boolean] = (col.untyped < lit(other)(col.uencoder).untyped).typed + def <=(other: U): TypedColumn[Boolean] = (col.untyped <= lit(other)(col.uencoder).untyped).typed + def >(other: U): TypedColumn[Boolean] = (col.untyped > lit(other)(col.uencoder).untyped).typed + def >=(other: U): TypedColumn[Boolean] = (col.untyped >= lit(other)(col.uencoder).untyped).typed } } diff --git a/dataset/src/main/scala/frameless/TypedDataset.scala b/dataset/src/main/scala/frameless/TypedDataset.scala index 992c752bf..c3b17d412 100644 --- a/dataset/src/main/scala/frameless/TypedDataset.scala +++ b/dataset/src/main/scala/frameless/TypedDataset.scala @@ -2,10 +2,8 @@ package frameless import frameless.ops._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, CreateStruct, EqualTo} -import org.apache.spark.sql.catalyst.plans.logical.{Join, Project} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter} import org.apache.spark.sql._ import shapeless._ import shapeless.ops.hlist.{Prepend, ToTraversable, Tupler} @@ -28,7 +26,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * * apache/spark */ - def agg[A](ca: TypedAggregate[T, A]): TypedDataset[A] = { + def agg[A](ca: TypedAggregate[A]): TypedDataset[A] = { implicit val ea = ca.uencoder val tuple1: TypedDataset[Tuple1[A]] = aggMany(ca) @@ -53,8 +51,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * apache/spark */ def agg[A, B]( - ca: TypedAggregate[T, A], - cb: TypedAggregate[T, B] + ca: TypedAggregate[A], + cb: TypedAggregate[B] ): TypedDataset[(A, B)] = { implicit val (ea, eb) = (ca.uencoder, cb.uencoder) aggMany(ca, cb) @@ -65,9 +63,9 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * apache/spark */ def agg[A, B, C]( - ca: TypedAggregate[T, A], - cb: TypedAggregate[T, B], - cc: TypedAggregate[T, C] + ca: TypedAggregate[A], + cb: TypedAggregate[B], + cc: TypedAggregate[C] ): TypedDataset[(A, B, C)] = { implicit val (ea, eb, ec) = (ca.uencoder, cb.uencoder, cc.uencoder) aggMany(ca, cb, cc) @@ -78,10 +76,10 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * apache/spark */ def agg[A, B, C, D]( - ca: TypedAggregate[T, A], - cb: TypedAggregate[T, B], - cc: TypedAggregate[T, C], - cd: TypedAggregate[T, D] + ca: TypedAggregate[A], + cb: TypedAggregate[B], + cc: TypedAggregate[C], + cd: TypedAggregate[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) @@ -95,7 +93,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val def applyProduct[U <: HList, Out0 <: HList, Out](columns: U)( implicit tc: AggregateTypes.Aux[T, U, Out0], - toTraversable: ToTraversable.Aux[U, List, UntypedExpression[T]], + toTraversable: ToTraversable.Aux[U, List, UntypedExpression], tupler: Tupler.Aux[Out0, Out], encoder: TypedEncoder[Out] ): TypedDataset[Out] = { @@ -153,7 +151,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val implicit exists: TypedColumn.Exists[T, column.T, A], encoder: TypedEncoder[A] - ): TypedColumn[T, A] = col(column) + ): TypedColumn[A] = col(column) /** Returns `TypedColumn` of type `A` given it's name. * @@ -167,9 +165,9 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val implicit exists: TypedColumn.Exists[T, column.T, A], encoder: TypedEncoder[A] - ): TypedColumn[T, A] = { + ): TypedColumn[A] = { val colExpr = dataset.col(column.value.name).as[A](TypedExpressionEncoder[A]) - new TypedColumn[T, A](colExpr) + new TypedColumn[A](colExpr) } object colMany extends SingletonProductArgs { @@ -178,11 +176,11 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val existsAll: TypedColumn.ExistsMany[T, U, Out], encoder: TypedEncoder[Out], toTraversable: ToTraversable.Aux[U, List, Symbol] - ): TypedColumn[T, Out] = { + ): TypedColumn[Out] = { val names = toTraversable(columns).map(_.name) val colExpr = FramelessInternals.resolveExpr(dataset, names) - new TypedColumn[T, Out](colExpr) + new TypedColumn[Out](colExpr) } } @@ -244,11 +242,11 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** 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 + * Differs from `TypedDatasetForward#filter` by taking a `TypedColumn[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] = { + def filter(column: TypedColumn[Boolean]): TypedDataset[T] = { val filtered = dataset.toDF() .filter(column.untyped) .as[T](TypedExpressionEncoder[T]) @@ -285,24 +283,27 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } def groupBy[K1]( - c1: TypedColumn[T, K1] + c1: TypedColumn[K1] ): GroupedBy1Ops[K1, T] = new GroupedBy1Ops[K1, T](this, c1) def groupBy[K1, K2]( - c1: TypedColumn[T, K1], - c2: TypedColumn[T, K2] + c1: TypedColumn[K1], + c2: TypedColumn[K2] ): GroupedBy2Ops[K1, K2, T] = new GroupedBy2Ops[K1, K2, T](this, c1, c2) object groupByMany extends ProductArgs { def applyProduct[TK <: HList, K <: HList, KT](groupedBy: TK)( implicit - ct: ColumnTypes.Aux[T, TK, K], + ct: ColumnTypes.Aux[TK, K], tupler: Tupler.Aux[K, KT], - toTraversable: ToTraversable.Aux[TK, List, UntypedExpression[T]] + toTraversable: ToTraversable.Aux[TK, List, UntypedExpression] ): GroupedByManyOps[T, TK, K, KT] = new GroupedByManyOps[T, TK, K, KT](self, groupedBy) } /** Fixes SPARK-6231, for more details see original code in [[Dataset#join]] **/ + import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, CreateStruct, EqualTo} + import org.apache.spark.sql.catalyst.plans.logical.{Join, Project} + import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter} private def resolveSelfJoin(join: Join): Join = { val plan = FramelessInternals.ofRows(dataset.sparkSession, join).queryExecution.analyzed.asInstanceOf[Join] val hasConflict = plan.left.output.intersect(plan.right.output).nonEmpty @@ -331,8 +332,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val def join[A, B]( right: TypedDataset[A], - leftCol: TypedColumn[T, B], - rightCol: TypedColumn[A, B] + leftCol: TypedColumn[B], + rightCol: TypedColumn[B] ): TypedDataset[(T, A)] = { implicit def re = right.encoder @@ -357,8 +358,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val def joinLeft[A: TypedEncoder, B]( right: TypedDataset[A], - leftCol: TypedColumn[T, B], - rightCol: TypedColumn[A, B] + leftCol: TypedColumn[B], + rightCol: TypedColumn[B] )(implicit e: TypedEncoder[(T, Option[A])]): TypedDataset[(T, Option[A])] = { val leftPlan = FramelessInternals.logicalPlan(dataset) val rightPlan = FramelessInternals.logicalPlan(right.dataset) @@ -379,34 +380,34 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val TypedDataset.create[(T, Option[A])](joinedDs) } - /** Takes a function from A => R and converts it to a UDF for TypedColumn[T, A] => TypedColumn[T, R]. + /** Takes a function from A => R and converts it to a UDF for TypedColumn[A] => TypedColumn[R]. */ def makeUDF[A: TypedEncoder, R: TypedEncoder](f: A => R): - TypedColumn[T, A] => TypedColumn[T, R] = functions.udf(f) + TypedColumn[A] => TypedColumn[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]. + * (TypedColumn[A1], TypedColumn[A2]) => TypedColumn[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) + (TypedColumn[A1], TypedColumn[A2]) => TypedColumn[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]. + * (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3]) => TypedColumn[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) + (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3]) => TypedColumn[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]. + * (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4]) => TypedColumn[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) + (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4]) => TypedColumn[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]. + * (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4], TypedColumn[A5]) => TypedColumn[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) + (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4], TypedColumn[A5]) => TypedColumn[R] = functions.udf(f) /** Type-safe projection from type T to Tuple1[A] * {{{ @@ -414,7 +415,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A]( - ca: TypedColumn[T, A] + ca: TypedColumn[A] ): TypedDataset[A] = { implicit val ea = ca.uencoder @@ -443,8 +444,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B] + ca: TypedColumn[A], + cb: TypedColumn[B] ): TypedDataset[(A, B)] = { implicit val (ea,eb) = (ca.uencoder, cb.uencoder) @@ -457,9 +458,9 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B, C]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C] + ca: TypedColumn[A], + cb: TypedColumn[B], + cc: TypedColumn[C] ): TypedDataset[(A, B, C)] = { implicit val (ea, eb, ec) = (ca.uencoder, cb.uencoder, cc.uencoder) @@ -472,10 +473,10 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B, C, D]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D] + ca: TypedColumn[A], + cb: TypedColumn[B], + cc: TypedColumn[C], + cd: TypedColumn[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) @@ -487,11 +488,11 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ 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] + ca: TypedColumn[A], + cb: TypedColumn[B], + cc: TypedColumn[C], + cd: TypedColumn[D], + ce: TypedColumn[E] ): TypedDataset[(A, B, C, D, E)] = { implicit val (ea, eb, ec, ed, ee) = (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder) @@ -505,12 +506,12 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ 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] + ca: TypedColumn[A], + cb: TypedColumn[B], + cc: TypedColumn[C], + cd: TypedColumn[D], + ce: TypedColumn[E], + cf: TypedColumn[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) @@ -524,13 +525,13 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ 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] + ca: TypedColumn[A], + cb: TypedColumn[B], + cc: TypedColumn[C], + cd: TypedColumn[D], + ce: TypedColumn[E], + cf: TypedColumn[F], + cg: TypedColumn[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) @@ -544,14 +545,14 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ 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] + ca: TypedColumn[A], + cb: TypedColumn[B], + cc: TypedColumn[C], + cd: TypedColumn[D], + ce: TypedColumn[E], + cf: TypedColumn[F], + cg: TypedColumn[G], + ch: TypedColumn[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) @@ -565,15 +566,15 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ 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] + ca: TypedColumn[A], + cb: TypedColumn[B], + cc: TypedColumn[C], + cd: TypedColumn[D], + ce: TypedColumn[E], + cf: TypedColumn[F], + cg: TypedColumn[G], + ch: TypedColumn[H], + ci: TypedColumn[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) @@ -587,16 +588,16 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ 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] + ca: TypedColumn[A], + cb: TypedColumn[B], + cc: TypedColumn[C], + cd: TypedColumn[D], + ce: TypedColumn[E], + cf: TypedColumn[F], + cg: TypedColumn[G], + ch: TypedColumn[H], + ci: TypedColumn[I], + cj: TypedColumn[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) @@ -606,8 +607,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val object selectMany extends ProductArgs { def applyProduct[U <: HList, Out0 <: HList, Out](columns: U)( implicit - ct: ColumnTypes.Aux[T, U, Out0], - toTraversable: ToTraversable.Aux[U, List, UntypedExpression[T]], + ct: ColumnTypes.Aux[U, Out0], + toTraversable: ToTraversable.Aux[U, List, UntypedExpression], tupler: Tupler.Aux[Out0, Out], encoder: TypedEncoder[Out] ): TypedDataset[Out] = { @@ -627,7 +628,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * val fNew: TypedDataset[(Int,Int,Boolean)] = f.withColumn(f('j) === 10) * }}} */ - def withColumn[A: TypedEncoder, H <: HList, FH <: HList, Out](ca: TypedColumn[T, A])( + def withColumn[A: TypedEncoder, H <: HList, FH <: HList, Out](ca: TypedColumn[A])( implicit genOfA: Generic.Aux[T, H], init: Prepend.Aux[H, A :: HNil, FH], diff --git a/dataset/src/main/scala/frameless/functions/AggregateFunctions.scala b/dataset/src/main/scala/frameless/functions/AggregateFunctions.scala index cdfc6a3f3..6ffad03b5 100644 --- a/dataset/src/main/scala/frameless/functions/AggregateFunctions.scala +++ b/dataset/src/main/scala/frameless/functions/AggregateFunctions.scala @@ -12,13 +12,13 @@ trait AggregateFunctions { * * apache/spark */ - def lit[A: TypedEncoder, T](value: A): TypedColumn[T, A] = frameless.functions.lit(value) + def lit[A: TypedEncoder](value: A): TypedColumn[A] = frameless.functions.lit(value) /** Aggregate function: returns the number of items in a group. * * apache/spark */ - def count[T](): TypedAggregate[T, Long] = { + def count(): TypedAggregate[Long] = { new TypedAggregate(untyped.count(untyped.lit(1))) } @@ -26,22 +26,22 @@ trait AggregateFunctions { * * apache/spark */ - def count[T](column: TypedColumn[T, _]): TypedAggregate[T, Long] = { - new TypedAggregate[T, Long](untyped.count(column.untyped)) + def count(column: TypedColumn[_]): TypedAggregate[Long] = { + new TypedAggregate[Long](untyped.count(column.untyped)) } /** Aggregate function: returns the number of distinct items in a group. * * apache/spark */ - def countDistinct[T](column: TypedColumn[T, _]): TypedAggregate[T, Long] = { - new TypedAggregate[T, Long](untyped.countDistinct(column.untyped)) + def countDistinct(column: TypedColumn[_]): TypedAggregate[Long] = { + new TypedAggregate[Long](untyped.countDistinct(column.untyped)) } /** Aggregate function: returns the approximate number of distinct items in a group. */ - def approxCountDistinct[T](column: TypedColumn[T, _]): TypedAggregate[T, Long] = { - new TypedAggregate[T, Long](untyped.approx_count_distinct(column.untyped)) + def approxCountDistinct(column: TypedColumn[_]): TypedAggregate[Long] = { + new TypedAggregate[Long](untyped.approx_count_distinct(column.untyped)) } /** Aggregate function: returns the approximate number of distinct items in a group. @@ -50,68 +50,68 @@ trait AggregateFunctions { * * apache/spark */ - def approxCountDistinct[T](column: TypedColumn[T, _], rsd: Double): TypedAggregate[T, Long] = { - new TypedAggregate[T, Long](untyped.approx_count_distinct(column.untyped, rsd)) + def approxCountDistinct(column: TypedColumn[_], rsd: Double): TypedAggregate[Long] = { + new TypedAggregate[Long](untyped.approx_count_distinct(column.untyped, rsd)) } /** Aggregate function: returns a list of objects with duplicates. * * apache/spark */ - def collectList[T, A: TypedEncoder](column: TypedColumn[T, A]): TypedAggregate[T, Vector[A]] = { - new TypedAggregate[T, Vector[A]](untyped.collect_list(column.untyped)) + def collectList[A: TypedEncoder](column: TypedColumn[A]): TypedAggregate[Vector[A]] = { + new TypedAggregate[Vector[A]](untyped.collect_list(column.untyped)) } /** Aggregate function: returns a set of objects with duplicate elements eliminated. * * apache/spark */ - def collectSet[T, A: TypedEncoder](column: TypedColumn[T, A]): TypedAggregate[T, Vector[A]] = { - new TypedAggregate[T, Vector[A]](untyped.collect_set(column.untyped)) + def collectSet[A: TypedEncoder](column: TypedColumn[A]): TypedAggregate[Vector[A]] = { + new TypedAggregate[Vector[A]](untyped.collect_set(column.untyped)) } /** Aggregate function: returns the sum of all values in the given column. * * apache/spark */ - def sum[A, T, Out](column: TypedColumn[T, A])( + def sum[A, Out](column: TypedColumn[A])( implicit summable: CatalystSummable[A, Out], oencoder: TypedEncoder[Out] - ): TypedAggregate[T, Out] = { + ): TypedAggregate[Out] = { val zeroExpr = Literal.create(summable.zero, TypedEncoder[Out].targetDataType) val sumExpr = expr(untyped.sum(column.untyped)) val sumOrZero = Coalesce(Seq(sumExpr, zeroExpr)) - new TypedAggregate[T, Out](sumOrZero) + new TypedAggregate[Out](sumOrZero) } /** Aggregate function: returns the sum of distinct values in the column. * * apache/spark */ - def sumDistinct[A, T, Out](column: TypedColumn[T, A])( + def sumDistinct[A, Out](column: TypedColumn[A])( implicit summable: CatalystSummable[A, Out], oencoder: TypedEncoder[Out] - ): TypedAggregate[T, Out] = { + ): TypedAggregate[Out] = { val zeroExpr = Literal.create(summable.zero, TypedEncoder[Out].targetDataType) val sumExpr = expr(untyped.sumDistinct(column.untyped)) val sumOrZero = Coalesce(Seq(sumExpr, zeroExpr)) - new TypedAggregate[T, Out](sumOrZero) + new TypedAggregate[Out](sumOrZero) } /** Aggregate function: returns the average of the values in a group. * * apache/spark */ - def avg[A, T, Out](column: TypedColumn[T, A])( + def avg[A, Out](column: TypedColumn[A])( implicit averageable: CatalystAverageable[A, Out], oencoder: TypedEncoder[Out] - ): TypedAggregate[T, Out] = { - new TypedAggregate[T, Out](untyped.avg(column.untyped)) + ): TypedAggregate[Out] = { + new TypedAggregate[Out](untyped.avg(column.untyped)) } @@ -122,8 +122,8 @@ trait AggregateFunctions { * * apache/spark */ - def variance[A: CatalystVariance, T](column: TypedColumn[T, A]): TypedAggregate[T, Double] = { - new TypedAggregate[T, Double](untyped.variance(column.untyped)) + def variance[A: CatalystVariance](column: TypedColumn[A]): TypedAggregate[Double] = { + new TypedAggregate[Double](untyped.variance(column.untyped)) } /** Aggregate function: returns the sample standard deviation. @@ -133,8 +133,8 @@ trait AggregateFunctions { * * apache/spark */ - def stddev[A: CatalystVariance, T](column: TypedColumn[T, A]): TypedAggregate[T, Double] = { - new TypedAggregate[T, Double](untyped.stddev(column.untyped)) + def stddev[A: CatalystVariance](column: TypedColumn[A]): TypedAggregate[Double] = { + new TypedAggregate[Double](untyped.stddev(column.untyped)) } /** @@ -145,13 +145,13 @@ trait AggregateFunctions { * * apache/spark */ - def stddevPop[A, T](column: TypedColumn[T, A])( + def stddevPop[A](column: TypedColumn[A])( implicit evCanBeDoubleA: CatalystCast[A, Double] - ): TypedAggregate[T, Option[Double]] = { + ): TypedAggregate[Option[Double]] = { implicit val c1 = column.uencoder - new TypedAggregate[T, Option[Double]]( + new TypedAggregate[Option[Double]]( untyped.stddev_pop(column.cast[Double].untyped) ) } @@ -164,13 +164,13 @@ trait AggregateFunctions { * * apache/spark */ - def stddevSamp[A, T](column: TypedColumn[T, A])( + def stddevSamp[A](column: TypedColumn[A])( implicit evCanBeDoubleA: CatalystCast[A, Double] - ): TypedAggregate[T, Option[Double]] = { + ): TypedAggregate[Option[Double]] = { implicit val c1 = column.uencoder - new TypedAggregate[T, Option[Double]]( + new TypedAggregate[Option[Double]]( untyped.stddev_samp(column.cast[Double].untyped) ) } @@ -179,18 +179,18 @@ trait AggregateFunctions { * * apache/spark */ - def max[A: CatalystOrdered, T](column: TypedColumn[T, A]): TypedAggregate[T, A] = { + def max[A: CatalystOrdered](column: TypedColumn[A]): TypedAggregate[A] = { implicit val c = column.uencoder - new TypedAggregate[T, A](untyped.max(column.untyped)) + new TypedAggregate[A](untyped.max(column.untyped)) } /** Aggregate function: returns the minimum value of the column in a group. * * apache/spark */ - def min[A: CatalystOrdered, T](column: TypedColumn[T, A]): TypedAggregate[T, A] = { + def min[A: CatalystOrdered](column: TypedColumn[A]): TypedAggregate[A] = { implicit val c = column.uencoder - new TypedAggregate[T, A](untyped.min(column.untyped)) + new TypedAggregate[A](untyped.min(column.untyped)) } /** Aggregate function: returns the first value in a group. @@ -200,9 +200,9 @@ trait AggregateFunctions { * * apache/spark */ - def first[A, T](column: TypedColumn[T, A]): TypedAggregate[T, A] = { + def first[A](column: TypedColumn[A]): TypedAggregate[A] = { implicit val c = column.uencoder - new TypedAggregate[T, A](untyped.first(column.untyped)) + new TypedAggregate[A](untyped.first(column.untyped)) } /** @@ -213,9 +213,9 @@ trait AggregateFunctions { * * apache/spark */ - def last[A, T](column: TypedColumn[T, A]): TypedAggregate[T, A] = { + def last[A](column: TypedColumn[A]): TypedAggregate[A] = { implicit val c = column.uencoder - new TypedAggregate[T, A](untyped.last(column.untyped)) + new TypedAggregate[A](untyped.last(column.untyped)) } /** @@ -226,15 +226,15 @@ trait AggregateFunctions { * * apache/spark */ - def corr[A, B, T](column1: TypedColumn[T, A], column2: TypedColumn[T, B])( + def corr[A, B](column1: TypedColumn[A], column2: TypedColumn[B])( implicit evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] - ): TypedAggregate[T, Option[Double]] = { + ): TypedAggregate[Option[Double]] = { implicit val c1 = column1.uencoder implicit val c2 = column2.uencoder - new TypedAggregate[T, Option[Double]]( + new TypedAggregate[Option[Double]]( untyped.corr(column1.cast[Double].untyped, column2.cast[Double].untyped) ) } @@ -247,15 +247,15 @@ trait AggregateFunctions { * * apache/spark */ - def covarPop[A, B, T](column1: TypedColumn[T, A], column2: TypedColumn[T, B])( + def covarPop[A, B](column1: TypedColumn[A], column2: TypedColumn[B])( implicit evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] - ): TypedAggregate[T, Option[Double]] = { + ): TypedAggregate[Option[Double]] = { implicit val c1 = column1.uencoder implicit val c2 = column2.uencoder - new TypedAggregate[T, Option[Double]]( + new TypedAggregate[Option[Double]]( untyped.covar_pop(column1.cast[Double].untyped, column2.cast[Double].untyped) ) } @@ -268,15 +268,15 @@ trait AggregateFunctions { * * apache/spark */ - def covarSamp[A, B, T](column1: TypedColumn[T, A], column2: TypedColumn[T, B])( + def covarSamp[A, B](column1: TypedColumn[A], column2: TypedColumn[B])( implicit evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] - ): TypedAggregate[T, Option[Double]] = { + ): TypedAggregate[Option[Double]] = { implicit val c1 = column1.uencoder implicit val c2 = column2.uencoder - new TypedAggregate[T, Option[Double]]( + new TypedAggregate[Option[Double]]( untyped.covar_samp(column1.cast[Double].untyped, column2.cast[Double].untyped) ) } @@ -290,13 +290,13 @@ trait AggregateFunctions { * * apache/spark */ - def kurtosis[A, T](column: TypedColumn[T, A])( + def kurtosis[A](column: TypedColumn[A])( implicit evCanBeDoubleA: CatalystCast[A, Double] - ): TypedAggregate[T, Option[Double]] = { + ): TypedAggregate[Option[Double]] = { implicit val c1 = column.uencoder - new TypedAggregate[T, Option[Double]]( + new TypedAggregate[Option[Double]]( untyped.kurtosis(column.cast[Double].untyped) ) } @@ -309,14 +309,14 @@ trait AggregateFunctions { * * apache/spark */ - def skewness[A, T](column: TypedColumn[T, A])( + def skewness[A](column: TypedColumn[A])( implicit evCanBeDoubleA: CatalystCast[A, Double] - ): TypedAggregate[T, Option[Double]] = { + ): TypedAggregate[Option[Double]] = { implicit val c1 = column.uencoder - new TypedAggregate[T, Option[Double]]( + new TypedAggregate[Option[Double]]( untyped.skewness(column.cast[Double].untyped) ) } -} \ No newline at end of file +} diff --git a/dataset/src/main/scala/frameless/functions/Udf.scala b/dataset/src/main/scala/frameless/functions/Udf.scala index 0b74d7a47..6e1256eab 100644 --- a/dataset/src/main/scala/frameless/functions/Udf.scala +++ b/dataset/src/main/scala/frameless/functions/Udf.scala @@ -20,10 +20,10 @@ trait Udf { * apache/spark */ def udf[T, A, R: TypedEncoder](f: A => R): - TypedColumn[T, A] => TypedColumn[T, R] = { + TypedColumn[A] => TypedColumn[R] = { u => val scalaUdf = FramelessUdf(f, List(u), TypedEncoder[R]) - new TypedColumn[T, R](scalaUdf) + new TypedColumn[R](scalaUdf) } /** Defines a user-defined function of 2 arguments as user-defined function (UDF). @@ -32,10 +32,10 @@ trait Udf { * apache/spark */ def udf[T, A1, A2, R: TypedEncoder](f: (A1,A2) => R): - (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R] = { + (TypedColumn[A1], TypedColumn[A2]) => TypedColumn[R] = { case us => - val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression[T]], TypedEncoder[R]) - new TypedColumn[T, R](scalaUdf) + val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression], TypedEncoder[R]) + new TypedColumn[R](scalaUdf) } /** Defines a user-defined function of 3 arguments as user-defined function (UDF). @@ -44,10 +44,10 @@ trait Udf { * apache/spark */ def udf[T, A1, A2, A3, R: TypedEncoder](f: (A1,A2,A3) => R): - (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3]) => TypedColumn[T, R] = { + (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3]) => TypedColumn[R] = { case us => - val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression[T]], TypedEncoder[R]) - new TypedColumn[T, R](scalaUdf) + val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression], TypedEncoder[R]) + new TypedColumn[R](scalaUdf) } /** Defines a user-defined function of 4 arguments as user-defined function (UDF). @@ -56,10 +56,10 @@ trait Udf { * apache/spark */ def udf[T, A1, A2, A3, A4, R: TypedEncoder](f: (A1,A2,A3,A4) => R): - (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4]) => TypedColumn[T, R] = { + (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4]) => TypedColumn[R] = { case us => - val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression[T]], TypedEncoder[R]) - new TypedColumn[T, R](scalaUdf) + val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression], TypedEncoder[R]) + new TypedColumn[R](scalaUdf) } /** Defines a user-defined function of 5 arguments as user-defined function (UDF). @@ -68,10 +68,10 @@ trait Udf { * apache/spark */ def udf[T, A1, A2, A3, A4, A5, 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] = { + (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4], TypedColumn[A5]) => TypedColumn[R] = { case us => - val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression[T]], TypedEncoder[R]) - new TypedColumn[T, R](scalaUdf) + val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression], TypedEncoder[R]) + new TypedColumn[R](scalaUdf) } } @@ -178,7 +178,7 @@ object FramelessUdf { // Spark needs case class with `children` field to mutate it def apply[T, R]( function: AnyRef, - cols: Seq[UntypedExpression[T]], + cols: Seq[UntypedExpression], rencoder: TypedEncoder[R] ): FramelessUdf[T, R] = FramelessUdf( function = function, diff --git a/dataset/src/main/scala/frameless/functions/UnaryFunctions.scala b/dataset/src/main/scala/frameless/functions/UnaryFunctions.scala index efce32925..e8e407ee8 100644 --- a/dataset/src/main/scala/frameless/functions/UnaryFunctions.scala +++ b/dataset/src/main/scala/frameless/functions/UnaryFunctions.scala @@ -10,24 +10,24 @@ trait UnaryFunctions { * * apache/spark */ - def size[T, A, V[_] : CatalystSizableCollection](column: TypedColumn[T, V[A]]): TypedColumn[T, Int] = - new TypedColumn[T, Int](implicitly[CatalystSizableCollection[V]].sizeOp(column.untyped)) + def size[T, A, V[_] : CatalystSizableCollection](column: TypedColumn[V[A]]): TypedColumn[Int] = + new TypedColumn[Int](implicitly[CatalystSizableCollection[V]].sizeOp(column.untyped)) /** Sorts the input array for the given column in ascending order, according to * the natural ordering of the array elements. * * apache/spark */ - def sortAscending[T, A: Ordering, V[_] : CatalystSortableCollection](column: TypedColumn[T, V[A]]): TypedColumn[T, V[A]] = - new TypedColumn[T, V[A]](implicitly[CatalystSortableCollection[V]].sortOp(column.untyped, sortAscending = true))(column.uencoder) + def sortAscending[T, A: Ordering, V[_] : CatalystSortableCollection](column: TypedColumn[V[A]]): TypedColumn[V[A]] = + new TypedColumn[V[A]](implicitly[CatalystSortableCollection[V]].sortOp(column.untyped, sortAscending = true))(column.uencoder) /** Sorts the input array for the given column in descending order, according to * the natural ordering of the array elements. * * apache/spark */ - def sortDescending[T, A: Ordering, V[_] : CatalystSortableCollection](column: TypedColumn[T, V[A]]): TypedColumn[T, V[A]] = - new TypedColumn[T, V[A]](implicitly[CatalystSortableCollection[V]].sortOp(column.untyped, sortAscending = false))(column.uencoder) + def sortDescending[T, A: Ordering, V[_] : CatalystSortableCollection](column: TypedColumn[V[A]]): TypedColumn[V[A]] = + new TypedColumn[V[A]](implicitly[CatalystSortableCollection[V]].sortOp(column.untyped, sortAscending = false))(column.uencoder) /** Creates a new row for each element in the given collection. The column types @@ -35,8 +35,8 @@ trait UnaryFunctions { * * apache/spark */ - def explode[T, A: TypedEncoder, V[_] : CatalystExplodableCollection](column: TypedColumn[T, V[A]]): TypedColumn[T, A] = - new TypedColumn[T, A](sparkFunctions.explode(column.untyped)) + def explode[T, A: TypedEncoder, V[_] : CatalystExplodableCollection](column: TypedColumn[V[A]]): TypedColumn[A] = + new TypedColumn[A](sparkFunctions.explode(column.untyped)) } trait CatalystSizableCollection[V[_]] { @@ -63,4 +63,4 @@ object CatalystSortableCollection { implicit def sortableVector: CatalystSortableCollection[Vector] = new CatalystSortableCollection[Vector] { def sortOp(col: Column, sortAscending: Boolean): Column = sparkFunctions.sort_array(col, sortAscending) } -} \ No newline at end of file +} diff --git a/dataset/src/main/scala/frameless/functions/package.scala b/dataset/src/main/scala/frameless/functions/package.scala index ccecdb495..51350621b 100644 --- a/dataset/src/main/scala/frameless/functions/package.scala +++ b/dataset/src/main/scala/frameless/functions/package.scala @@ -6,7 +6,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal package object functions extends Udf with UnaryFunctions { object aggregate extends AggregateFunctions - def lit[A: TypedEncoder, T](value: A): TypedColumn[T, A] = { + def lit[A: TypedEncoder, T](value: A): TypedColumn[A] = { val encoder = TypedEncoder[A] if (ScalaReflection.isNativeType(encoder.sourceDataType) && encoder.targetDataType == encoder.sourceDataType) { diff --git a/dataset/src/main/scala/frameless/implicits.scala b/dataset/src/main/scala/frameless/implicits.scala index f70c5f042..32ca3a5ee 100644 --- a/dataset/src/main/scala/frameless/implicits.scala +++ b/dataset/src/main/scala/frameless/implicits.scala @@ -4,32 +4,32 @@ object implicits { object widen { // frameless prefixed to avoid implicit name collision - implicit def framelessByteToShort[T](col: TypedColumn[T, Byte]): TypedColumn[T, Short] = col.cast[Short] - implicit def framelessByteToInt[T](col: TypedColumn[T, Byte]): TypedColumn[T, Int] = col.cast[Int] - implicit def framelessByteToLong[T](col: TypedColumn[T, Byte]): TypedColumn[T, Long] = col.cast[Long] - implicit def framelessByteToDouble[T](col: TypedColumn[T, Byte]): TypedColumn[T, Double] = col.cast[Double] - implicit def framelessByteToBigDecimal[T](col: TypedColumn[T, Byte]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] + implicit def framelessByteToShort[T](col: TypedColumn[Byte]): TypedColumn[Short] = col.cast[Short] + implicit def framelessByteToInt[T](col: TypedColumn[Byte]): TypedColumn[Int] = col.cast[Int] + implicit def framelessByteToLong[T](col: TypedColumn[Byte]): TypedColumn[Long] = col.cast[Long] + implicit def framelessByteToDouble[T](col: TypedColumn[Byte]): TypedColumn[Double] = col.cast[Double] + implicit def framelessByteToBigDecimal[T](col: TypedColumn[Byte]): TypedColumn[BigDecimal] = col.cast[BigDecimal] - implicit def framelessShortToInt[T](col: TypedColumn[T, Short]): TypedColumn[T, Int] = col.cast[Int] - implicit def framelessShortToLong[T](col: TypedColumn[T, Short]): TypedColumn[T, Long] = col.cast[Long] - implicit def framelessShortToDouble[T](col: TypedColumn[T, Short]): TypedColumn[T, Double] = col.cast[Double] - implicit def framelessShortToBigDecimal[T](col: TypedColumn[T, Short]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] + implicit def framelessShortToInt[T](col: TypedColumn[Short]): TypedColumn[Int] = col.cast[Int] + implicit def framelessShortToLong[T](col: TypedColumn[Short]): TypedColumn[Long] = col.cast[Long] + implicit def framelessShortToDouble[T](col: TypedColumn[Short]): TypedColumn[Double] = col.cast[Double] + implicit def framelessShortToBigDecimal[T](col: TypedColumn[Short]): TypedColumn[BigDecimal] = col.cast[BigDecimal] - implicit def framelessIntToLong[T](col: TypedColumn[T, Int]): TypedColumn[T, Long] = col.cast[Long] - implicit def framelessIntToDouble[T](col: TypedColumn[T, Int]): TypedColumn[T, Double] = col.cast[Double] - implicit def framelessIntToBigDecimal[T](col: TypedColumn[T, Int]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] + implicit def framelessIntToLong[T](col: TypedColumn[Int]): TypedColumn[Long] = col.cast[Long] + implicit def framelessIntToDouble[T](col: TypedColumn[Int]): TypedColumn[Double] = col.cast[Double] + implicit def framelessIntToBigDecimal[T](col: TypedColumn[Int]): TypedColumn[BigDecimal] = col.cast[BigDecimal] - implicit def framelessLongToDouble[T](col: TypedColumn[T, Long]): TypedColumn[T, Double] = col.cast[Double] - implicit def framelessLongToBigDecimal[T](col: TypedColumn[T, Long]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] + implicit def framelessLongToDouble[T](col: TypedColumn[Long]): TypedColumn[Double] = col.cast[Double] + implicit def framelessLongToBigDecimal[T](col: TypedColumn[Long]): TypedColumn[BigDecimal] = col.cast[BigDecimal] - implicit def framelessDoubleToBigDecimal[T](col: TypedColumn[T, Double]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] + implicit def framelessDoubleToBigDecimal[T](col: TypedColumn[Double]): TypedColumn[BigDecimal] = col.cast[BigDecimal] // we don't have floats yet, but then this is lawful (or not?): // - // implicit def byteToFloat[T](col: TypedColumn[T, Byte]): TypedColumn[T, Float] = col.cast[Float] - // implicit def intToFloat[T](col: TypedColumn[T, Int]): TypedColumn[T, Float] = col.cast[Float] - // implicit def longToFloat[T](col: TypedColumn[T, Long]): TypedColumn[T, Float] = col.cast[Float] - // implicit def floatToDouble[T](col: TypedColumn[T, Float]): TypedColumn[T, Double] = col.cast[Double] - // implicit def floatToBigDecimal[T](col: TypedColumn[T, Float]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] + // implicit def byteToFloat[T](col: TypedColumn[Byte]): TypedColumn[Float] = col.cast[Float] + // implicit def intToFloat[T](col: TypedColumn[Int]): TypedColumn[Float] = col.cast[Float] + // implicit def longToFloat[T](col: TypedColumn[Long]): TypedColumn[Float] = col.cast[Float] + // implicit def floatToDouble[T](col: TypedColumn[Float]): TypedColumn[Double] = col.cast[Double] + // implicit def floatToBigDecimal[T](col: TypedColumn[Float]): TypedColumn[BigDecimal] = col.cast[BigDecimal] } } diff --git a/dataset/src/main/scala/frameless/ops/AggregateTypes.scala b/dataset/src/main/scala/frameless/ops/AggregateTypes.scala index 403c25301..bc38be616 100644 --- a/dataset/src/main/scala/frameless/ops/AggregateTypes.scala +++ b/dataset/src/main/scala/frameless/ops/AggregateTypes.scala @@ -23,6 +23,6 @@ object AggregateTypes { implicit def deriveCons1[T, H, TT <: HList, V <: HList]( implicit tail: AggregateTypes.Aux[T, TT, V] - ): AggregateTypes.Aux[T, TypedAggregate[T, H] :: TT, H :: V] = - new AggregateTypes[T, TypedAggregate[T, H] :: TT] {type Out = H :: V} + ): AggregateTypes.Aux[T, TypedAggregate[H] :: TT, H :: V] = + new AggregateTypes[T, TypedAggregate[H] :: TT] {type Out = H :: V} } diff --git a/dataset/src/main/scala/frameless/ops/ColumnTypes.scala b/dataset/src/main/scala/frameless/ops/ColumnTypes.scala index e5ae6aea2..b0606febe 100644 --- a/dataset/src/main/scala/frameless/ops/ColumnTypes.scala +++ b/dataset/src/main/scala/frameless/ops/ColumnTypes.scala @@ -8,21 +8,21 @@ import shapeless._ * @note This type class is mostly a workaround to issue with slow implicit derivation for Comapped. * @example * {{{ - * type U = TypedColumn[T,A] :: TypedColumn[T,B] :: TypedColumn[T,C] :: HNil + * type U = TypedColumn[A] :: TypedColumn[B] :: TypedColumn[C] :: HNil * type Out = A :: B :: C :: HNil * }}} */ -trait ColumnTypes[T, U <: HList] { +trait ColumnTypes[U <: HList] { type Out <: HList } object ColumnTypes { - type Aux[T, U <: HList, Out0 <: HList] = ColumnTypes[T, U] {type Out = Out0} + type Aux[U <: HList, Out0 <: HList] = ColumnTypes[U] {type Out = Out0} - implicit def deriveHNil[T]: ColumnTypes.Aux[T, HNil, HNil] = new ColumnTypes[T, HNil] { type Out = HNil } + implicit def deriveHNil: ColumnTypes.Aux[HNil, HNil] = new ColumnTypes[HNil] { type Out = HNil } - implicit def deriveCons[T, H, TT <: HList, V <: HList]( - implicit tail: ColumnTypes.Aux[T, TT, V] - ): ColumnTypes.Aux[T, TypedColumn[T, H] :: TT, H :: V] = - new ColumnTypes[T, TypedColumn[T, H] :: TT] {type Out = H :: V} + implicit def deriveCons[H, TT <: HList, V <: HList]( + implicit tail: ColumnTypes.Aux[TT, V] + ): ColumnTypes.Aux[TypedColumn[H] :: TT, H :: V] = + new ColumnTypes[TypedColumn[H] :: TT] {type Out = H :: V} } diff --git a/dataset/src/main/scala/frameless/ops/GroupByOps.scala b/dataset/src/main/scala/frameless/ops/GroupByOps.scala index a217a3216..1aa7b691e 100644 --- a/dataset/src/main/scala/frameless/ops/GroupByOps.scala +++ b/dataset/src/main/scala/frameless/ops/GroupByOps.scala @@ -12,8 +12,8 @@ class GroupedByManyOps[T, TK <: HList, K <: HList, KT]( groupedBy: TK )( implicit - ct: ColumnTypes.Aux[T, TK, K], - toTraversable: ToTraversable.Aux[TK, List, UntypedExpression[T]], + ct: ColumnTypes.Aux[TK, K], + toTraversable: ToTraversable.Aux[TK, List, UntypedExpression], tupler: Tupler.Aux[K, KT] ) { @@ -24,10 +24,10 @@ class GroupedByManyOps[T, TK <: HList, K <: HList, KT]( append: Prepend.Aux[K, C, Out0], toTuple: Tupler.Aux[Out0, Out1], encoder: TypedEncoder[Out1], - columnsToList: ToTraversable.Aux[TC, List, UntypedExpression[T]] + columnsToList: ToTraversable.Aux[TC, List, UntypedExpression] ): TypedDataset[Out1] = { - def expr(c: UntypedExpression[T]): Column = new Column(c.expr) + def expr(c: UntypedExpression): Column = new Column(c.expr) val groupByExprs = toTraversable(groupedBy).map(expr) val aggregates = @@ -93,32 +93,32 @@ object GroupedByManyOps { class GroupedBy1Ops[K1, V]( self: TypedDataset[V], - g1: TypedColumn[V, K1] + g1: TypedColumn[K1] ) { private def underlying = new GroupedByManyOps(self, g1 :: HNil) private implicit def eg1 = g1.uencoder - def agg[U1](c1: TypedAggregate[V, U1]): TypedDataset[(K1, U1)] = { + def agg[U1](c1: TypedAggregate[U1]): TypedDataset[(K1, U1)] = { implicit val e1 = c1.uencoder underlying.agg(c1) } - def agg[U1, U2](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2]): TypedDataset[(K1, U1, U2)] = { + def agg[U1, U2](c1: TypedAggregate[U1], c2: TypedAggregate[U2]): TypedDataset[(K1, U1, U2)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder underlying.agg(c1, c2) } - def agg[U1, U2, U3](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3]): TypedDataset[(K1, U1, U2, U3)] = { + def agg[U1, U2, U3](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3]): TypedDataset[(K1, U1, U2, U3)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder underlying.agg(c1, c2, c3) } - def agg[U1, U2, U3, U4](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3], c4: TypedAggregate[V, U4]): TypedDataset[(K1, U1, U2, U3, U4)] = { + def agg[U1, U2, U3, U4](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3], c4: TypedAggregate[U4]): TypedDataset[(K1, U1, U2, U3, U4)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder; implicit val e4 = c4.uencoder underlying.agg(c1, c2, c3, c4) } - def agg[U1, U2, U3, U4, U5](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3], c4: TypedAggregate[V, U4], c5: TypedAggregate[V, U5]): TypedDataset[(K1, U1, U2, U3, U4, U5)] = { + def agg[U1, U2, U3, U4, U5](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3], c4: TypedAggregate[U4], c5: TypedAggregate[U5]): TypedDataset[(K1, U1, U2, U3, U4, U5)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder; implicit val e4 = c4.uencoder; implicit val e5 = c5.uencoder underlying.agg(c1, c2, c3, c4, c5) } @@ -134,34 +134,34 @@ class GroupedBy1Ops[K1, V]( class GroupedBy2Ops[K1, K2, V]( self: TypedDataset[V], - g1: TypedColumn[V, K1], - g2: TypedColumn[V, K2] + g1: TypedColumn[K1], + g2: TypedColumn[K2] ) { private def underlying = new GroupedByManyOps(self, g1 :: g2 :: HNil) private implicit def eg1 = g1.uencoder private implicit def eg2 = g2.uencoder - def agg[U1](c1: TypedAggregate[V, U1]): TypedDataset[(K1, K2, U1)] = { + def agg[U1](c1: TypedAggregate[U1]): TypedDataset[(K1, K2, U1)] = { implicit val e1 = c1.uencoder underlying.agg(c1) } - def agg[U1, U2](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2]): TypedDataset[(K1, K2, U1, U2)] = { + def agg[U1, U2](c1: TypedAggregate[U1], c2: TypedAggregate[U2]): TypedDataset[(K1, K2, U1, U2)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder underlying.agg(c1, c2) } - def agg[U1, U2, U3](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3]): TypedDataset[(K1, K2, U1, U2, U3)] = { + def agg[U1, U2, U3](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3]): TypedDataset[(K1, K2, U1, U2, U3)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder underlying.agg(c1, c2, c3) } - def agg[U1, U2, U3, U4](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3], c4: TypedAggregate[V, U4]): TypedDataset[(K1, K2, U1, U2, U3, U4)] = { + def agg[U1, U2, U3, U4](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3], c4: TypedAggregate[U4]): TypedDataset[(K1, K2, U1, U2, U3, U4)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder; implicit val e4 = c4.uencoder underlying.agg(c1 , c2 , c3 , c4) } - def agg[U1, U2, U3, U4, U5](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3], c4: TypedAggregate[V, U4], c5: TypedAggregate[V, U5]): TypedDataset[(K1, K2, U1, U2, U3, U4, U5)] = { + def agg[U1, U2, U3, U4, U5](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3], c4: TypedAggregate[U4], c5: TypedAggregate[U5]): TypedDataset[(K1, K2, U1, U2, U3, U4, U5)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder; implicit val e4 = c4.uencoder; implicit val e5 = c5.uencoder underlying.agg(c1, c2, c3, c4, c5) } diff --git a/dataset/src/test/scala/frameless/ColTests.scala b/dataset/src/test/scala/frameless/ColTests.scala index ad62aa068..35aeae8c7 100644 --- a/dataset/src/test/scala/frameless/ColTests.scala +++ b/dataset/src/test/scala/frameless/ColTests.scala @@ -33,10 +33,10 @@ class ColTests extends TypedDatasetSuite { type X2X2 = X2[X2[Int, String], X2[Long, Boolean]] val x2x2 = TypedDataset.create[X2X2](Nil) - val aa: TypedColumn[X2X2, Int] = x2x2.colMany('a, 'a) - val ab: TypedColumn[X2X2, String] = x2x2.colMany('a, 'b) - val ba: TypedColumn[X2X2, Long] = x2x2.colMany('b, 'a) - val bb: TypedColumn[X2X2, Boolean] = x2x2.colMany('b, 'b) + val aa: TypedColumn[Int] = x2x2.colMany('a, 'a) + val ab: TypedColumn[String] = x2x2.colMany('a, 'b) + val ba: TypedColumn[Long] = x2x2.colMany('b, 'a) + val bb: TypedColumn[Boolean] = x2x2.colMany('b, 'b) illTyped("x2x2.colMany('a, 'c)") illTyped("x2x2.colMany('a, 'a, 'a)") diff --git a/dataset/src/test/scala/frameless/SelectTests.scala b/dataset/src/test/scala/frameless/SelectTests.scala index b6b9772c8..48c92f231 100644 --- a/dataset/src/test/scala/frameless/SelectTests.scala +++ b/dataset/src/test/scala/frameless/SelectTests.scala @@ -397,4 +397,4 @@ class SelectTests extends TypedDatasetSuite { val e = TypedDataset.create[(Int, String, Long)]((1, "a", 2L) :: (2, "b", 4L) :: (2, "b", 1L) :: Nil) illTyped("""e.select(frameless.functions.aggregate.sum(e('_1)))""") } -} \ No newline at end of file +} diff --git a/dataset/src/test/scala/frameless/WidenTests.scala b/dataset/src/test/scala/frameless/WidenTests.scala index 7b2eaa2fe..ee486aa14 100644 --- a/dataset/src/test/scala/frameless/WidenTests.scala +++ b/dataset/src/test/scala/frameless/WidenTests.scala @@ -16,7 +16,7 @@ class WidenTests extends TypedDatasetSuite { def widenSum[A: TypedEncoder: CatalystNumeric: Numeric, B: TypedEncoder](a: A, b: B)( implicit view: B => A, - colView: TypedColumn[X2[A, B], B] => TypedColumn[X2[A, B], A] + colView: TypedColumn[B] => TypedColumn[A] ): Prop = { val df = TypedDataset.create(X2(a, b) :: Nil) val sum = implicitly[Numeric[A]].plus(a, view(b)) @@ -31,7 +31,7 @@ class WidenTests extends TypedDatasetSuite { def widen[A: TypedEncoder, B: TypedEncoder](a: A)( implicit view: A => B, - colView: TypedColumn[X1[A], A] => TypedColumn[X1[A], B] + colView: TypedColumn[A] => TypedColumn[B] ): Prop = { val df = TypedDataset.create(X1(a) :: Nil) val got = df.select(colView(df.col('a))).collect().run() diff --git a/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala b/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala index 99729cd64..a70a3066b 100644 --- a/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala +++ b/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala @@ -23,7 +23,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { else falsified :| s"Expected $a but got $b, which is more than 1% off and greater than epsilon = $epsilon." } - def sparkSchema[A: TypedEncoder, U](f: TypedColumn[X1[A], A] => TypedAggregate[X1[A], U]): Prop = { + def sparkSchema[A: TypedEncoder, U](f: TypedColumn[A] => TypedAggregate[U]): Prop = { val df = TypedDataset.create[X1[A]](Nil) val col = f(df.col('a)) @@ -366,7 +366,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { xs: List[X3[Int, A, B]] ) ( - framelessFun: (TypedColumn[X3[Int, A, B], A], TypedColumn[X3[Int, A, B], B]) => TypedAggregate[X3[Int, A, B], Option[Double]], + framelessFun: (TypedColumn[A], TypedColumn[B]) => TypedAggregate[Option[Double]], sparkFun: (Column, Column) => Column ) ( @@ -405,7 +405,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { xs: List[X2[Int, A]] ) ( - framelessFun: (TypedColumn[X2[Int, A], A]) => TypedAggregate[X2[Int, A], Option[Double]], + framelessFun: (TypedColumn[A]) => TypedAggregate[Option[Double]], sparkFun: (Column) => Column ) ( @@ -447,7 +447,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { encEv: Encoder[(Int, A, B)], evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] - ): Prop = bivariatePropTemplate(xs)(corr[A,B,X3[Int, A, B]],org.apache.spark.sql.functions.corr) + ): Prop = bivariatePropTemplate(xs)(corr[A, B],org.apache.spark.sql.functions.corr) check(forAll(prop[Double, Double] _)) check(forAll(prop[Double, Int] _)) @@ -466,7 +466,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] ): Prop = bivariatePropTemplate(xs)( - covarPop[A, B, X3[Int, A, B]], + covarPop[A, B], org.apache.spark.sql.functions.covar_pop ) @@ -487,7 +487,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] ): Prop = bivariatePropTemplate(xs)( - covarSamp[A, B, X3[Int, A, B]], + covarSamp[A, B], org.apache.spark.sql.functions.covar_samp ) @@ -507,7 +507,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { encEv: Encoder[(Int, A)], evCanBeDoubleA: CatalystCast[A, Double] ): Prop = univariatePropTemplate(xs)( - kurtosis[A, X2[Int, A]], + kurtosis[A], org.apache.spark.sql.functions.kurtosis ) @@ -527,7 +527,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { encEv: Encoder[(Int, A)], evCanBeDoubleA: CatalystCast[A, Double] ): Prop = univariatePropTemplate(xs)( - skewness[A, X2[Int, A]], + skewness[A], org.apache.spark.sql.functions.skewness ) @@ -547,7 +547,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { encEv: Encoder[(Int, A)], evCanBeDoubleA: CatalystCast[A, Double] ): Prop = univariatePropTemplate(xs)( - stddevPop[A, X2[Int, A]], + stddevPop[A], org.apache.spark.sql.functions.stddev_pop ) @@ -567,7 +567,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { encEv: Encoder[(Int, A)], evCanBeDoubleA: CatalystCast[A, Double] ): Prop = univariatePropTemplate(xs)( - stddevSamp[A, X2[Int, A]], + stddevSamp[A], org.apache.spark.sql.functions.stddev_samp ) check(forAll(prop[Double] _)) diff --git a/dataset/src/test/scala/frameless/functions/UnaryFunctionsTest.scala b/dataset/src/test/scala/frameless/functions/UnaryFunctionsTest.scala index d50d6c021..17fe99c4c 100644 --- a/dataset/src/test/scala/frameless/functions/UnaryFunctionsTest.scala +++ b/dataset/src/test/scala/frameless/functions/UnaryFunctionsTest.scala @@ -69,4 +69,4 @@ class UnaryFunctionsTest extends TypedDatasetSuite { check(forAll(prop[Char] _)) check(forAll(prop[String] _)) } -} \ No newline at end of file +} diff --git a/dataset/src/test/scala/frameless/ops/ColumnTypesTest.scala b/dataset/src/test/scala/frameless/ops/ColumnTypesTest.scala index 303eb2cbd..f40ebf7c6 100644 --- a/dataset/src/test/scala/frameless/ops/ColumnTypesTest.scala +++ b/dataset/src/test/scala/frameless/ops/ColumnTypesTest.scala @@ -12,12 +12,12 @@ class ColumnTypesTest extends TypedDatasetSuite { val d: TypedDataset[X4[A, B, C, D]] = TypedDataset.create(data) val hlist = d('a) :: d('b) :: d('c) :: d('d) :: HNil - type TC[N] = TypedColumn[X4[A,B,C,D], N] + type TC[N] = TypedColumn[N] type IN = TC[A] :: TC[B] :: TC[C] :: TC[D] :: HNil type OUT = A :: B :: C :: D :: HNil - implicitly[ColumnTypes.Aux[X4[A,B,C,D], IN, OUT]] + implicitly[ColumnTypes.Aux[IN, OUT]] Prop.passed // successful compilation implies test correctness } From 0a1afb39904be8a9fdbd52591b0d110f792075a1 Mon Sep 17 00:00:00 2001 From: Olivier Blanvillain Date: Mon, 7 Aug 2017 17:05:00 +0200 Subject: [PATCH 2/6] Add all joins with a TypedColumn[Boolean] condition --- .../main/scala/frameless/TypedDataset.scala | 133 ++++++++------- .../apache/spark/sql/FramelessInternals.scala | 21 ++- .../src/test/scala/frameless/JoinTests.scala | 152 +++++++++++++++++- 3 files changed, 240 insertions(+), 66 deletions(-) diff --git a/dataset/src/main/scala/frameless/TypedDataset.scala b/dataset/src/main/scala/frameless/TypedDataset.scala index c3b17d412..48400000a 100644 --- a/dataset/src/main/scala/frameless/TypedDataset.scala +++ b/dataset/src/main/scala/frameless/TypedDataset.scala @@ -2,7 +2,9 @@ package frameless import frameless.ops._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute, Literal} +import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter, RightOuter, FullOuter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql._ import shapeless._ @@ -300,10 +302,83 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val ): GroupedByManyOps[T, TK, K, KT] = new GroupedByManyOps[T, TK, K, KT](self, groupedBy) } + /** 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[Boolean]) + (implicit e: TypedEncoder[(T, U)]): TypedDataset[(T, U)] = { + import FramelessInternals._ + val leftPlan = logicalPlan(dataset) + val rightPlan = logicalPlan(other.dataset) + val join = resolveSelfJoin(Join(leftPlan, rightPlan, Inner, Some(condition.expr))) + val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) + val joinedDs = mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(T, U)]) + TypedDataset.create[(T, U)](joinedDs) + } + + /** 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[Boolean]) + (implicit e: TypedEncoder[(Option[T], Option[U])]): TypedDataset[(Option[T], Option[U])] = { + import FramelessInternals._ + val leftPlan = logicalPlan(dataset) + val rightPlan = logicalPlan(other.dataset) + val join = resolveSelfJoin(Join(leftPlan, rightPlan, FullOuter, Some(condition.expr))) + val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) + val joinedDs = mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(Option[T], Option[U])]) + TypedDataset.create[(Option[T], Option[U])](joinedDs) + } + + /** 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[Boolean]) + (implicit e: TypedEncoder[(Option[T], U)]): TypedDataset[(Option[T], U)] = { + import FramelessInternals._ + val leftPlan = logicalPlan(dataset) + val rightPlan = logicalPlan(other.dataset) + val join = resolveSelfJoin(Join(leftPlan, rightPlan, RightOuter, Some(condition.expr))) + val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) + val joinedDs = mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(Option[T], U)]) + TypedDataset.create[(Option[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[Boolean]) + (implicit e: TypedEncoder[(T, Option[U])]): TypedDataset[(T, Option[U])] = { + import FramelessInternals._ + val leftPlan = logicalPlan(dataset) + val rightPlan = logicalPlan(other.dataset) + val join = resolveSelfJoin(Join(leftPlan, rightPlan, LeftOuter, Some(condition.expr))) + val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) + val joinedDs = mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(T, Option[U])]) + + TypedDataset.create[(T, Option[U])](joinedDs) + } + + /** 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[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[Boolean]): TypedDataset[T] = + new TypedDataset(self.dataset.join(other.dataset, condition.untyped, "leftanti") + .as[T](TypedExpressionEncoder(encoder))) + /** Fixes SPARK-6231, for more details see original code in [[Dataset#join]] **/ - import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, CreateStruct, EqualTo} - import org.apache.spark.sql.catalyst.plans.logical.{Join, Project} - import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter} private def resolveSelfJoin(join: Join): Join = { val plan = FramelessInternals.ofRows(dataset.sparkSession, join).queryExecution.analyzed.asInstanceOf[Join] val hasConflict = plan.left.output.intersect(plan.right.output).nonEmpty @@ -330,56 +405,6 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } } - def join[A, B]( - right: TypedDataset[A], - leftCol: TypedColumn[B], - rightCol: TypedColumn[B] - ): TypedDataset[(T, A)] = { - implicit def re = right.encoder - - val leftPlan = FramelessInternals.logicalPlan(dataset) - val rightPlan = FramelessInternals.logicalPlan(right.dataset) - val condition = EqualTo(leftCol.expr, rightCol.expr) - - val join = resolveSelfJoin(Join(leftPlan, rightPlan, Inner, Some(condition))) - val joined = FramelessInternals.executePlan(dataset, join) - val leftOutput = joined.analyzed.output.take(leftPlan.output.length) - val rightOutput = joined.analyzed.output.takeRight(rightPlan.output.length) - - val joinedPlan = Project(List( - Alias(CreateStruct(leftOutput), "_1")(), - Alias(CreateStruct(rightOutput), "_2")() - ), joined.analyzed) - - val joinedDs = FramelessInternals.mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(T, A)]) - - TypedDataset.create[(T, A)](joinedDs) - } - - def joinLeft[A: TypedEncoder, B]( - right: TypedDataset[A], - leftCol: TypedColumn[B], - rightCol: TypedColumn[B] - )(implicit e: TypedEncoder[(T, Option[A])]): TypedDataset[(T, Option[A])] = { - val leftPlan = FramelessInternals.logicalPlan(dataset) - val rightPlan = FramelessInternals.logicalPlan(right.dataset) - val condition = EqualTo(leftCol.expr, rightCol.expr) - - val join = resolveSelfJoin(Join(leftPlan, rightPlan, LeftOuter, Some(condition))) - val joined = FramelessInternals.executePlan(dataset, join) - val leftOutput = joined.analyzed.output.take(leftPlan.output.length) - val rightOutput = joined.analyzed.output.takeRight(rightPlan.output.length) - - val joinedPlan = Project(List( - Alias(CreateStruct(leftOutput), "_1")(), - Alias(CreateStruct(rightOutput), "_2")() - ), joined.analyzed) - - val joinedDs = FramelessInternals.mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(T, Option[A])]) - - TypedDataset.create[(T, Option[A])](joinedDs) - } - /** Takes a function from A => R and converts it to a UDF for TypedColumn[A] => TypedColumn[R]. */ def makeUDF[A: TypedEncoder, R: TypedEncoder](f: A => R): diff --git a/dataset/src/main/scala/org/apache/spark/sql/FramelessInternals.scala b/dataset/src/main/scala/org/apache/spark/sql/FramelessInternals.scala index 934546db5..7eb0c171b 100644 --- a/dataset/src/main/scala/org/apache/spark/sql/FramelessInternals.scala +++ b/dataset/src/main/scala/org/apache/spark/sql/FramelessInternals.scala @@ -1,7 +1,8 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.expressions.{Alias, CreateStruct} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.types.ObjectType @@ -23,17 +24,25 @@ object FramelessInternals { def logicalPlan(ds: Dataset[_]): LogicalPlan = ds.logicalPlan - def executePlan(ds: Dataset[_], plan: LogicalPlan): QueryExecution = { + def executePlan(ds: Dataset[_], plan: LogicalPlan): QueryExecution = ds.sparkSession.sessionState.executePlan(plan) + + def joinPlan(ds: Dataset[_], plan: LogicalPlan, leftPlan: LogicalPlan, rightPlan: LogicalPlan): LogicalPlan = { + val joined = executePlan(ds, plan) + val leftOutput = joined.analyzed.output.take(leftPlan.output.length) + val rightOutput = joined.analyzed.output.takeRight(rightPlan.output.length) + + Project(List( + Alias(CreateStruct(leftOutput), "_1")(), + Alias(CreateStruct(rightOutput), "_2")() + ), joined.analyzed) } - def mkDataset[T](sqlContext: SQLContext, plan: LogicalPlan, encoder: Encoder[T]): Dataset[T] = { + def mkDataset[T](sqlContext: SQLContext, plan: LogicalPlan, encoder: Encoder[T]): Dataset[T] = new Dataset(sqlContext, plan, encoder) - } - def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = { + def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = Dataset.ofRows(sparkSession, logicalPlan) - } // because org.apache.spark.sql.types.UserDefinedType is private[spark] type UserDefinedType[A >: Null] = org.apache.spark.sql.types.UserDefinedType[A] diff --git a/dataset/src/test/scala/frameless/JoinTests.scala b/dataset/src/test/scala/frameless/JoinTests.scala index 5bf65027d..5ada0a903 100644 --- a/dataset/src/test/scala/frameless/JoinTests.scala +++ b/dataset/src/test/scala/frameless/JoinTests.scala @@ -4,7 +4,7 @@ import org.scalacheck.Prop import org.scalacheck.Prop._ class JoinTests extends TypedDatasetSuite { - test("ab.joinLeft(ac, ab.a, ac.a)") { + test("ab.joinRight(ac)(ab.a == ac.a)") { def prop[ A : TypedEncoder : Ordering, B : TypedEncoder : Ordering, @@ -13,7 +13,62 @@ class JoinTests extends TypedDatasetSuite { val leftDs = TypedDataset.create(left) val rightDs = TypedDataset.create(right) val joinedDs = leftDs - .joinLeft(rightDs, leftDs.col('a), rightDs.col('a)) + .joinRight(rightDs)(leftDs.col('a) === rightDs.col('a)) + .collect().run().toVector.sorted + + val leftKeys = left.map(_.a).toSet + val joined = { + for { + ab <- left + ac <- right if ac.a == ab.a + } yield (Some(ab), ac) + }.toVector ++ { + for { + ac <- right if !leftKeys.contains(ac.a) + } yield (None, ac) + }.toVector + + (joined.sorted ?= joinedDs) && (joinedDs.map(_._2).toSet ?= right.toSet) + } + + check(forAll(prop[Int, Long, String] _)) + } + + test("ab.joinInner(ac)(ab.a == ac.a)") { + def prop[ + A : TypedEncoder : Ordering, + B : TypedEncoder : Ordering, + C : TypedEncoder : Ordering + ](left: List[X2[A, B]], right: List[X2[A, C]]): Prop = { + val leftDs = TypedDataset.create(left) + val rightDs = TypedDataset.create(right) + val joinedDs = leftDs + .joinInner(rightDs)(leftDs.col('a) === rightDs.col('a)) + .collect().run().toVector.sorted + + val joined = { + for { + ab <- left + ac <- right if ac.a == ab.a + } yield (ab, ac) + }.toVector + + joined.sorted ?= joinedDs + } + + check(forAll(prop[Int, Long, String] _)) + } + + test("ab.joinLeft(ac)(ab.a == ac.a)") { + def prop[ + A : TypedEncoder : Ordering, + B : TypedEncoder : Ordering, + C : TypedEncoder : Ordering + ](left: List[X2[A, B]], right: List[X2[A, C]]): Prop = { + val leftDs = TypedDataset.create(left) + val rightDs = TypedDataset.create(right) + val joinedDs = leftDs + .joinLeft(rightDs)(leftDs.col('a) === rightDs.col('a)) .collect().run().toVector.sorted val rightKeys = right.map(_.a).toSet @@ -34,7 +89,7 @@ class JoinTests extends TypedDatasetSuite { check(forAll(prop[Int, Long, String] _)) } - test("ab.join(ac, ab.a, ac.a)") { + test("ab.joinFull(ac)(ab.a == ac.a)") { def prop[ A : TypedEncoder : Ordering, B : TypedEncoder : Ordering, @@ -43,23 +98,108 @@ class JoinTests extends TypedDatasetSuite { val leftDs = TypedDataset.create(left) val rightDs = TypedDataset.create(right) val joinedDs = leftDs - .join(rightDs, leftDs.col('a), rightDs.col('a)) + .joinFull(rightDs)(leftDs.col('a) === rightDs.col('a)) .collect().run().toVector.sorted + val rightKeys = right.map(_.a).toSet + val leftKeys = left.map(_.a).toSet val joined = { for { ab <- left ac <- right if ac.a == ab.a + } yield (Some(ab), Some(ac)) + }.toVector ++ { + for { + ab <- left if !rightKeys.contains(ab.a) + } yield (Some(ab), None) + }.toVector ++ { + for { + ac <- right if !leftKeys.contains(ac.a) + } yield (None, Some(ac)) + }.toVector + + (joined.sorted ?= joinedDs) + } + + check(forAll(prop[Int, Long, String] _)) + } + + test("ab.joinCross(ac)") { + def prop[ + A : TypedEncoder : Ordering, + B : TypedEncoder : Ordering, + C : TypedEncoder : Ordering + ](left: List[X2[A, B]], right: List[X2[A, C]]): Prop = { + val leftDs = TypedDataset.create(left) + val rightDs = TypedDataset.create(right) + val joinedDs = leftDs + .joinCross(rightDs) + .collect().run().toVector.sorted + + val joined = { + for { + ab <- left + ac <- right } yield (ab, ac) }.toVector + (joined.sorted ?= joinedDs) + } + + check(forAll(prop[Int, Long, String] _)) + } + + test("ab.joinLeftSemi(ac)(ab.a == ac.a)") { + def prop[ + A : TypedEncoder : Ordering, + B : TypedEncoder : Ordering, + C : TypedEncoder : Ordering + ](left: List[X2[A, B]], right: List[X2[A, C]]): Prop = { + val leftDs = TypedDataset.create(left) + val rightDs = TypedDataset.create(right) + val rightKeys = right.map(_.a).toSet + val joinedDs = leftDs + .joinLeftSemi(rightDs)(leftDs.col('a) === rightDs.col('a)) + .collect().run().toVector.sorted + + val joined = { + for { + ab <- left if rightKeys.contains(ab.a) + } yield ab + }.toVector + + joined.sorted ?= joinedDs + } + + check(forAll(prop[Int, Long, String] _)) + } + + test("ab.joinLeftAnti(ac)(ab.a == ac.a)") { + def prop[ + A : TypedEncoder : Ordering, + B : TypedEncoder : Ordering, + C : TypedEncoder : Ordering + ](left: List[X2[A, B]], right: List[X2[A, C]]): Prop = { + val leftDs = TypedDataset.create(left) + val rightDs = TypedDataset.create(right) + val rightKeys = right.map(_.a).toSet + val joinedDs = leftDs + .joinLeftAnti(rightDs)(leftDs.col('a) === rightDs.col('a)) + .collect().run().toVector.sorted + + val joined = { + for { + ab <- left if !rightKeys.contains(ab.a) + } yield ab + }.toVector + joined.sorted ?= joinedDs } check(forAll(prop[Int, Long, String] _)) } - test("self join") { + test("self inner join") { def prop[ A : TypedEncoder : Ordering, B : TypedEncoder : Ordering @@ -68,7 +208,7 @@ class JoinTests extends TypedDatasetSuite { val count = ds.dataset.join(ds.dataset, ds.dataset.col("a") === ds.dataset.col("a")).count() - val countDs = ds.join(ds, ds.col('a), ds.col('a)) + val countDs = ds.joinInner(ds)(ds.col('a) === ds.col('a)) .count().run() count ?= countDs From 71e67da04703e0b988c2e52035dd8d614f955442 Mon Sep 17 00:00:00 2001 From: Olivier Blanvillain Date: Tue, 8 Aug 2017 11:54:56 +0200 Subject: [PATCH 3/6] Fix tut --- docs/src/main/tut/FeatureOverview.md | 42 ++++++++++++++-------------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/docs/src/main/tut/FeatureOverview.md b/docs/src/main/tut/FeatureOverview.md index c829e320c..59062c5f2 100644 --- a/docs/src/main/tut/FeatureOverview.md +++ b/docs/src/main/tut/FeatureOverview.md @@ -84,7 +84,7 @@ aptDs.select('citi) aptTypedDs.select(aptTypedDs('surface) * 10, aptTypedDs('surface) + 2).show().run() ``` -Note that unlike the standard Spark API where some operations are lazy and some are not, **TypedDatasets have all operations to be lazy.** +Note that unlike the standard Spark API where some operations are lazy and some are not, **TypedDatasets have all operations to be lazy.** In the above example, `show()` is lazy. It requires to apply `run()` for the `show` job to materialize. A more detailed explanation of `Job` is given [here](Job.md). @@ -94,9 +94,9 @@ Next we compute the price by surface unit: val priceBySurfaceUnit = aptTypedDs.select(aptTypedDs('price) / aptTypedDs('surface)) ``` -As the error suggests, we can't divide a `TypedColumn` of `Double` by `Int.` -For safety, in Frameless only math operations between same types is allowed. -There are two ways to proceed here: +As the error suggests, we can't divide a `TypedColumn` of `Double` by `Int.` +For safety, in Frameless only math operations between same types is allowed. +There are two ways to proceed here: (a) Explicitly cast `Int` to `Double` (manual) @@ -121,20 +121,20 @@ Let's try to cast a `TypedColumn` of `String` to `Double`: aptTypedDs('city).cast[Double] ``` -The compile-time error tells us that to perform the cast, an evidence -(in the form of `CatalystCast[String, Double]`) must be available. -Since casting from `String` to `Double` is not allowed, this results -in a compilation error. +The compile-time error tells us that to perform the cast, an evidence +(in the form of `CatalystCast[String, Double]`) must be available. +Since casting from `String` to `Double` is not allowed, this results +in a compilation error. -Check [here](https://github.com/typelevel/frameless/blob/master/core/src/main/scala/frameless/CatalystCast.scala) +Check [here](https://github.com/typelevel/frameless/blob/master/core/src/main/scala/frameless/CatalystCast.scala) for the set of available `CatalystCast.` ## TypeSafe TypedDataset casting and projections With `select()` the resulting TypedDataset is of type `TypedDataset[TupleN[...]]` (with N in `[1...10]`). For example, if we select three columns with types `String`, `Int`, and `Boolean` the result will have type -`TypedDataset[(String, Int, Boolean)]`. To select more than ten columns use the `selectMany()` method. -Select has better IDE support than the macro based selectMany, so prefer `select()` for the general case. +`TypedDataset[(String, Int, Boolean)]`. To select more than ten columns use the `selectMany()` method. +Select has better IDE support than the macro based selectMany, so prefer `select()` for the general case. We often want to give more expressive types to the result of our computations. `as[T]` allows us to safely cast a `TypedDataset[U]` to another of type `TypedDataset[T]` as long @@ -259,23 +259,23 @@ cityPriceRatio.groupBy(cityPriceRatio('_1)).agg(avg(cityPriceRatio('_2))).show() ### Entire TypedDataset Aggregation We often want to aggregate the entire `TypedDataset` and skip the `groupBy()` clause. -In `Frameless` you can do this using the `agg()` operator directly on the `TypedDataset`. -In the following example, we compute the average price, the average surface, -the minimum surface, and the set of cities for the entire dataset. +In `Frameless` you can do this using the `agg()` operator directly on the `TypedDataset`. +In the following example, we compute the average price, the average surface, +the minimum surface, and the set of cities for the entire dataset. ```tut:book case class Stats( - avgPrice: Double, - avgSurface: Double, - minSurface: Int, + avgPrice: Double, + avgSurface: Double, + minSurface: Int, allCities: Vector[String]) - + aptds.agg( - avg(aptds('price)), + avg(aptds('price)), avg(aptds('surface)), min(aptds('surface)), collectSet(aptds('city)) -).as[Stats].show().run() +).as[Stats].show().run() ``` ## Joins @@ -295,7 +295,7 @@ val citiInfoTypedDS = TypedDataset.create(cityInfo) Here is how to join the population information to the apartment's dataset. ```tut:book -val withCityInfo = aptTypedDs.join(citiInfoTypedDS, aptTypedDs('city), citiInfoTypedDS('name)) +val withCityInfo = aptTypedDs.joinInner(citiInfoTypedDS)(aptTypedDs('city) === citiInfoTypedDS('name)) withCityInfo.show().run() ``` From bcabdb2d4cea16db676eb13a5bd3e5f6ec92acbd Mon Sep 17 00:00:00 2001 From: Olivier Blanvillain Date: Sat, 16 Sep 2017 00:49:15 +0200 Subject: [PATCH 4/6] Revert "Remove TypedColumn's first type parameter" This reverts commit eb66711c00ee526e49c60459290ffb708c7296ca. Conflicts: dataset/src/main/scala/frameless/TypedDataset.scala --- build.sbt | 1 - .../scala/frameless/FramelessSyntax.scala | 2 +- .../main/scala/frameless/TypedColumn.scala | 73 +++---- .../main/scala/frameless/TypedDataset.scala | 183 +++++++++--------- .../functions/AggregateFunctions.scala | 114 +++++------ .../main/scala/frameless/functions/Udf.scala | 30 +-- .../frameless/functions/UnaryFunctions.scala | 18 +- .../scala/frameless/functions/package.scala | 2 +- .../src/main/scala/frameless/implicits.scala | 40 ++-- .../scala/frameless/ops/AggregateTypes.scala | 4 +- .../scala/frameless/ops/ColumnTypes.scala | 16 +- .../main/scala/frameless/ops/GroupByOps.scala | 34 ++-- .../src/test/scala/frameless/ColTests.scala | 8 +- .../test/scala/frameless/SelectTests.scala | 2 +- .../src/test/scala/frameless/WidenTests.scala | 4 +- .../functions/AggregateFunctionsTests.scala | 20 +- .../functions/UnaryFunctionsTest.scala | 2 +- .../scala/frameless/ops/ColumnTypesTest.scala | 4 +- 18 files changed, 278 insertions(+), 279 deletions(-) diff --git a/build.sbt b/build.sbt index 7f61a0d85..9abc5d38e 100644 --- a/build.sbt +++ b/build.sbt @@ -68,7 +68,6 @@ lazy val commonScalacOptions = Seq( "-feature", "-unchecked", "-Xfatal-warnings", - // "-Xlog-implicits", "-Xlint:-missing-interpolator,_", "-Yinline-warnings", "-Yno-adapted-args", diff --git a/dataset/src/main/scala/frameless/FramelessSyntax.scala b/dataset/src/main/scala/frameless/FramelessSyntax.scala index a3d38a76f..fa102200e 100644 --- a/dataset/src/main/scala/frameless/FramelessSyntax.scala +++ b/dataset/src/main/scala/frameless/FramelessSyntax.scala @@ -4,7 +4,7 @@ import org.apache.spark.sql.{Column, DataFrame, Dataset} trait FramelessSyntax { implicit class ColumnSyntax(self: Column) { - def typed[T, U: TypedEncoder]: TypedColumn[U] = new TypedColumn[U](self) + def typed[T, U: TypedEncoder]: TypedColumn[T, U] = new TypedColumn[T, U](self) } implicit class DatasetSyntax[T: TypedEncoder](self: Dataset[T]) { diff --git a/dataset/src/main/scala/frameless/TypedColumn.scala b/dataset/src/main/scala/frameless/TypedColumn.scala index 9c0009e44..61a1b1114 100644 --- a/dataset/src/main/scala/frameless/TypedColumn.scala +++ b/dataset/src/main/scala/frameless/TypedColumn.scala @@ -10,7 +10,7 @@ import shapeless._ import scala.annotation.implicitNotFound -sealed trait UntypedExpression { +sealed trait UntypedExpression[T] { def expr: Expression def uencoder: TypedEncoder[_] override def toString: String = expr.toString() @@ -22,13 +22,14 @@ sealed trait UntypedExpression { * at https://github.com/apache/spark, licensed under Apache v2.0 available at * http://www.apache.org/licenses/LICENSE-2.0 * + * @tparam T type of dataset * @tparam U type of column */ -sealed class TypedColumn[U]( +sealed class TypedColumn[T, U]( val expr: Expression)( implicit val uencoder: TypedEncoder[U] -) extends UntypedExpression { self => +) extends UntypedExpression[T] { self => /** From an untyped Column to a [[TypedColumn]] * @@ -45,7 +46,7 @@ sealed class TypedColumn[U]( private def withExpr(newExpr: Expression): Column = new Column(newExpr) - private def equalsTo(other: TypedColumn[U]): TypedColumn[Boolean] = withExpr { + private def equalsTo(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = withExpr { if (uencoder.nullable && uencoder.targetDataType.typeName != "struct") EqualNullSafe(self.expr, other.expr) else EqualTo(self.expr, other.expr) }.typed @@ -57,7 +58,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def ===(other: U): TypedColumn[Boolean] = equalsTo(lit(other)) + def ===(other: U): TypedColumn[T, Boolean] = equalsTo(lit(other)) /** Equality test. * {{{ @@ -66,7 +67,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def ===(other: TypedColumn[U]): TypedColumn[Boolean] = equalsTo(other) + def ===(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = equalsTo(other) /** Inequality test. * {{{ @@ -75,7 +76,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def =!=(other: TypedColumn[U]): TypedColumn[Boolean] = withExpr { + def =!=(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = withExpr { Not(equalsTo(other).expr) }.typed @@ -86,7 +87,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def =!=(other: U): TypedColumn[Boolean] = withExpr { + def =!=(other: U): TypedColumn[T, Boolean] = withExpr { Not(equalsTo(lit(other)).expr) }.typed @@ -94,14 +95,14 @@ sealed class TypedColumn[U]( * * apache/spark */ - def isNone(implicit isOption: U <:< Option[_]): TypedColumn[Boolean] = - equalsTo(lit(None.asInstanceOf[U])) + def isNone(implicit isOption: U <:< Option[_]): TypedColumn[T, Boolean] = + equalsTo(lit[U,T](None.asInstanceOf[U])) /** True if the current expression is an Option and it's not None. * * apache/spark */ - def isNotNone(implicit isOption: U <:< Option[_]): TypedColumn[Boolean] = withExpr { + def isNotNone(implicit isOption: U <:< Option[_]): TypedColumn[T, Boolean] = withExpr { Not(equalsTo(lit(None.asInstanceOf[U])).expr) }.typed @@ -113,7 +114,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def plus(other: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = + def plus(other: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = self.untyped.plus(other.untyped).typed /** Sum of this expression and another expression. @@ -124,7 +125,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def +(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = plus(u) + def +(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = plus(u) /** Sum of this expression (column) with a constant. * {{{ @@ -135,7 +136,7 @@ sealed class TypedColumn[U]( * @param u a constant of the same type * apache/spark */ - def +(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[U] = self.untyped.plus(u).typed + def +(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = self.untyped.plus(u).typed /** Unary minus, i.e. negate the expression. * {{{ @@ -145,7 +146,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def unary_-(implicit n: CatalystNumeric[U]): TypedColumn[U] = (-self.untyped).typed + def unary_-(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = (-self.untyped).typed /** Subtraction. Subtract the other expression from this expression. * {{{ @@ -155,7 +156,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def minus(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = + def minus(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = self.untyped.minus(u.untyped).typed /** Subtraction. Subtract the other expression from this expression. @@ -166,7 +167,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def -(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = minus(u) + def -(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = minus(u) /** Subtraction. Subtract the other expression from this expression. * {{{ @@ -177,7 +178,7 @@ sealed class TypedColumn[U]( * @param u a constant of the same type * apache/spark */ - def -(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[U] = self.untyped.minus(u).typed + def -(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = self.untyped.minus(u).typed /** Multiplication of this expression and another expression. * {{{ @@ -187,7 +188,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def multiply(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = + def multiply(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = self.untyped.multiply(u.untyped).typed /** Multiplication of this expression and another expression. @@ -198,7 +199,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def *(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[U] = multiply(u) + def *(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = multiply(u) /** Multiplication of this expression a constant. * {{{ @@ -208,7 +209,7 @@ sealed class TypedColumn[U]( * * apache/spark */ - def *(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[U] = self.untyped.multiply(u).typed + def *(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[T, U] = self.untyped.multiply(u).typed /** * Division this expression by another expression. @@ -220,7 +221,7 @@ sealed class TypedColumn[U]( * @param u another column of the same type * apache/spark */ - def divide(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[Double] = + def divide(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, Double] = self.untyped.divide(u.untyped).typed /** @@ -233,7 +234,7 @@ sealed class TypedColumn[U]( * @param u another column of the same type * apache/spark */ - def /(u: TypedColumn[U])(implicit n: CatalystNumeric[U]): TypedColumn[Double] = divide(u) + def /(u: TypedColumn[T, U])(implicit n: CatalystNumeric[U]): TypedColumn[T, Double] = divide(u) /** * Division this expression by another expression. @@ -245,14 +246,14 @@ sealed class TypedColumn[U]( * @param u a constant of the same type * apache/spark */ - def /(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[Double] = self.untyped.divide(u).typed + def /(u: U)(implicit n: CatalystNumeric[U]): TypedColumn[T, Double] = self.untyped.divide(u).typed /** Casts the column to a different type. * {{{ * df.select(df('a).cast[Int]) * }}} */ - def cast[A: TypedEncoder](implicit c: CatalystCast[U, A]): TypedColumn[A] = + def cast[A: TypedEncoder](implicit c: CatalystCast[U, A]): TypedColumn[T, A] = self.untyped.cast(TypedEncoder[A].targetDataType).typed } @@ -261,10 +262,10 @@ sealed class TypedColumn[U]( * @tparam T type of dataset * @tparam U type of column for `groupBy` */ -sealed class TypedAggregate[U](val expr: Expression)( +sealed class TypedAggregate[T, U](val expr: Expression)( implicit val uencoder: TypedEncoder[U] -) extends UntypedExpression { +) extends UntypedExpression[T] { def this(column: Column)(implicit uenc: TypedEncoder[U]) { this(FramelessInternals.expr(column)) @@ -307,15 +308,15 @@ object TypedColumn { ): Exists[T, K, V] = new Exists[T, K, V] {} } - implicit class OrderedTypedColumnSyntax[U: CatalystOrdered](col: TypedColumn[U]) { - def <(other: TypedColumn[U]): TypedColumn[Boolean] = (col.untyped < other.untyped).typed - def <=(other: TypedColumn[U]): TypedColumn[Boolean] = (col.untyped <= other.untyped).typed - def >(other: TypedColumn[U]): TypedColumn[Boolean] = (col.untyped > other.untyped).typed - def >=(other: TypedColumn[U]): TypedColumn[Boolean] = (col.untyped >= other.untyped).typed + implicit class OrderedTypedColumnSyntax[T, U: CatalystOrdered](col: TypedColumn[T, U]) { + def <(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = (col.untyped < other.untyped).typed + def <=(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = (col.untyped <= other.untyped).typed + def >(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = (col.untyped > other.untyped).typed + def >=(other: TypedColumn[T, U]): TypedColumn[T, Boolean] = (col.untyped >= other.untyped).typed - def <(other: U): TypedColumn[Boolean] = (col.untyped < lit(other)(col.uencoder).untyped).typed - def <=(other: U): TypedColumn[Boolean] = (col.untyped <= lit(other)(col.uencoder).untyped).typed - def >(other: U): TypedColumn[Boolean] = (col.untyped > lit(other)(col.uencoder).untyped).typed - def >=(other: U): TypedColumn[Boolean] = (col.untyped >= lit(other)(col.uencoder).untyped).typed + def <(other: U): TypedColumn[T, Boolean] = (col.untyped < lit(other)(col.uencoder).untyped).typed + def <=(other: U): TypedColumn[T, Boolean] = (col.untyped <= lit(other)(col.uencoder).untyped).typed + def >(other: U): TypedColumn[T, Boolean] = (col.untyped > lit(other)(col.uencoder).untyped).typed + def >=(other: U): TypedColumn[T, Boolean] = (col.untyped >= lit(other)(col.uencoder).untyped).typed } } diff --git a/dataset/src/main/scala/frameless/TypedDataset.scala b/dataset/src/main/scala/frameless/TypedDataset.scala index 48400000a..77433ae6d 100644 --- a/dataset/src/main/scala/frameless/TypedDataset.scala +++ b/dataset/src/main/scala/frameless/TypedDataset.scala @@ -2,8 +2,7 @@ package frameless import frameless.ops._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute, Literal} -import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, CreateStruct, EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter, RightOuter, FullOuter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql._ @@ -28,7 +27,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * * apache/spark */ - def agg[A](ca: TypedAggregate[A]): TypedDataset[A] = { + def agg[A](ca: TypedAggregate[T, A]): TypedDataset[A] = { implicit val ea = ca.uencoder val tuple1: TypedDataset[Tuple1[A]] = aggMany(ca) @@ -53,8 +52,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * apache/spark */ def agg[A, B]( - ca: TypedAggregate[A], - cb: TypedAggregate[B] + ca: TypedAggregate[T, A], + cb: TypedAggregate[T, B] ): TypedDataset[(A, B)] = { implicit val (ea, eb) = (ca.uencoder, cb.uencoder) aggMany(ca, cb) @@ -65,9 +64,9 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * apache/spark */ def agg[A, B, C]( - ca: TypedAggregate[A], - cb: TypedAggregate[B], - cc: TypedAggregate[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) @@ -78,10 +77,10 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * apache/spark */ def agg[A, B, C, D]( - ca: TypedAggregate[A], - cb: TypedAggregate[B], - cc: TypedAggregate[C], - cd: TypedAggregate[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) aggMany(ca, cb, cc, cd) @@ -95,7 +94,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val def applyProduct[U <: HList, Out0 <: HList, Out](columns: U)( implicit tc: AggregateTypes.Aux[T, U, Out0], - toTraversable: ToTraversable.Aux[U, List, UntypedExpression], + toTraversable: ToTraversable.Aux[U, List, UntypedExpression[T]], tupler: Tupler.Aux[Out0, Out], encoder: TypedEncoder[Out] ): TypedDataset[Out] = { @@ -153,7 +152,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val implicit exists: TypedColumn.Exists[T, column.T, A], encoder: TypedEncoder[A] - ): TypedColumn[A] = col(column) + ): TypedColumn[T, A] = col(column) /** Returns `TypedColumn` of type `A` given it's name. * @@ -167,9 +166,9 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val implicit exists: TypedColumn.Exists[T, column.T, A], encoder: TypedEncoder[A] - ): TypedColumn[A] = { + ): TypedColumn[T, A] = { val colExpr = dataset.col(column.value.name).as[A](TypedExpressionEncoder[A]) - new TypedColumn[A](colExpr) + new TypedColumn[T, A](colExpr) } object colMany extends SingletonProductArgs { @@ -178,11 +177,11 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val existsAll: TypedColumn.ExistsMany[T, U, Out], encoder: TypedEncoder[Out], toTraversable: ToTraversable.Aux[U, List, Symbol] - ): TypedColumn[Out] = { + ): TypedColumn[T, Out] = { val names = toTraversable(columns).map(_.name) val colExpr = FramelessInternals.resolveExpr(dataset, names) - new TypedColumn[Out](colExpr) + new TypedColumn[T, Out](colExpr) } } @@ -244,11 +243,11 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** Returns a new [[frameless.TypedDataset]] that only contains elements where `column` is `true`. * - * Differs from `TypedDatasetForward#filter` by taking a `TypedColumn[Boolean]` instead of a + * 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[Boolean]): TypedDataset[T] = { + def filter(column: TypedColumn[T, Boolean]): TypedDataset[T] = { val filtered = dataset.toDF() .filter(column.untyped) .as[T](TypedExpressionEncoder[T]) @@ -285,20 +284,20 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } def groupBy[K1]( - c1: TypedColumn[K1] + c1: TypedColumn[T, K1] ): GroupedBy1Ops[K1, T] = new GroupedBy1Ops[K1, T](this, c1) def groupBy[K1, K2]( - c1: TypedColumn[K1], - c2: TypedColumn[K2] + c1: TypedColumn[T, K1], + c2: TypedColumn[T, K2] ): GroupedBy2Ops[K1, K2, T] = new GroupedBy2Ops[K1, K2, T](this, c1, c2) object groupByMany extends ProductArgs { def applyProduct[TK <: HList, K <: HList, KT](groupedBy: TK)( implicit - ct: ColumnTypes.Aux[TK, K], + ct: ColumnTypes.Aux[T, TK, K], tupler: Tupler.Aux[K, KT], - toTraversable: ToTraversable.Aux[TK, List, UntypedExpression] + toTraversable: ToTraversable.Aux[TK, List, UntypedExpression[T]] ): GroupedByManyOps[T, TK, K, KT] = new GroupedByManyOps[T, TK, K, KT](self, groupedBy) } @@ -408,31 +407,31 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** Takes a function from A => R and converts it to a UDF for TypedColumn[A] => TypedColumn[R]. */ def makeUDF[A: TypedEncoder, R: TypedEncoder](f: A => R): - TypedColumn[A] => TypedColumn[R] = functions.udf(f) + TypedColumn[T, A] => TypedColumn[T, R] = functions.udf(f) /** Takes a function from (A1, A2) => R and converts it to a UDF for - * (TypedColumn[A1], TypedColumn[A2]) => TypedColumn[R]. + * (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R]. */ def makeUDF[A1: TypedEncoder, A2: TypedEncoder, R: TypedEncoder](f: (A1, A2) => R): - (TypedColumn[A1], TypedColumn[A2]) => TypedColumn[R] = functions.udf(f) + (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[A1], TypedColumn[A2], TypedColumn[A3]) => TypedColumn[R]. + * (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[A1], TypedColumn[A2], TypedColumn[A3]) => TypedColumn[R] = functions.udf(f) + (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[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4]) => TypedColumn[R]. + * (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[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4]) => TypedColumn[R] = functions.udf(f) + (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[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4], TypedColumn[A5]) => TypedColumn[R]. + * (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[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4], TypedColumn[A5]) => TypedColumn[R] = functions.udf(f) + (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] * {{{ @@ -440,7 +439,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A]( - ca: TypedColumn[A] + ca: TypedColumn[T, A] ): TypedDataset[A] = { implicit val ea = ca.uencoder @@ -469,8 +468,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B]( - ca: TypedColumn[A], - cb: TypedColumn[B] + ca: TypedColumn[T, A], + cb: TypedColumn[T, B] ): TypedDataset[(A, B)] = { implicit val (ea,eb) = (ca.uencoder, cb.uencoder) @@ -483,9 +482,9 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B, C]( - ca: TypedColumn[A], - cb: TypedColumn[B], - cc: TypedColumn[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) @@ -498,10 +497,10 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B, C, D]( - ca: TypedColumn[A], - cb: TypedColumn[B], - cc: TypedColumn[C], - cd: TypedColumn[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) selectMany(ca, cb, cc, cd) @@ -513,11 +512,11 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B, C, D, E]( - ca: TypedColumn[A], - cb: TypedColumn[B], - cc: TypedColumn[C], - cd: TypedColumn[D], - ce: TypedColumn[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) @@ -531,12 +530,12 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B, C, D, E, F]( - ca: TypedColumn[A], - cb: TypedColumn[B], - cc: TypedColumn[C], - cd: TypedColumn[D], - ce: TypedColumn[E], - cf: TypedColumn[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) @@ -550,13 +549,13 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B, C, D, E, F, G]( - ca: TypedColumn[A], - cb: TypedColumn[B], - cc: TypedColumn[C], - cd: TypedColumn[D], - ce: TypedColumn[E], - cf: TypedColumn[F], - cg: TypedColumn[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) @@ -570,14 +569,14 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B, C, D, E, F, G, H]( - ca: TypedColumn[A], - cb: TypedColumn[B], - cc: TypedColumn[C], - cd: TypedColumn[D], - ce: TypedColumn[E], - cf: TypedColumn[F], - cg: TypedColumn[G], - ch: TypedColumn[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) @@ -591,15 +590,15 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B, C, D, E, F, G, H, I]( - ca: TypedColumn[A], - cb: TypedColumn[B], - cc: TypedColumn[C], - cd: TypedColumn[D], - ce: TypedColumn[E], - cf: TypedColumn[F], - cg: TypedColumn[G], - ch: TypedColumn[H], - ci: TypedColumn[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) @@ -613,16 +612,16 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * }}} */ def select[A, B, C, D, E, F, G, H, I, J]( - ca: TypedColumn[A], - cb: TypedColumn[B], - cc: TypedColumn[C], - cd: TypedColumn[D], - ce: TypedColumn[E], - cf: TypedColumn[F], - cg: TypedColumn[G], - ch: TypedColumn[H], - ci: TypedColumn[I], - cj: TypedColumn[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) @@ -632,8 +631,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val object selectMany extends ProductArgs { def applyProduct[U <: HList, Out0 <: HList, Out](columns: U)( implicit - ct: ColumnTypes.Aux[U, Out0], - toTraversable: ToTraversable.Aux[U, List, UntypedExpression], + ct: ColumnTypes.Aux[T, U, Out0], + toTraversable: ToTraversable.Aux[U, List, UntypedExpression[T]], tupler: Tupler.Aux[Out0, Out], encoder: TypedEncoder[Out] ): TypedDataset[Out] = { @@ -653,7 +652,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * val fNew: TypedDataset[(Int,Int,Boolean)] = f.withColumn(f('j) === 10) * }}} */ - def withColumn[A: TypedEncoder, H <: HList, FH <: HList, Out](ca: TypedColumn[A])( + def withColumn[A: TypedEncoder, H <: HList, FH <: HList, Out](ca: TypedColumn[T, A])( implicit genOfA: Generic.Aux[T, H], init: Prepend.Aux[H, A :: HNil, FH], diff --git a/dataset/src/main/scala/frameless/functions/AggregateFunctions.scala b/dataset/src/main/scala/frameless/functions/AggregateFunctions.scala index 6ffad03b5..cdfc6a3f3 100644 --- a/dataset/src/main/scala/frameless/functions/AggregateFunctions.scala +++ b/dataset/src/main/scala/frameless/functions/AggregateFunctions.scala @@ -12,13 +12,13 @@ trait AggregateFunctions { * * apache/spark */ - def lit[A: TypedEncoder](value: A): TypedColumn[A] = frameless.functions.lit(value) + def lit[A: TypedEncoder, T](value: A): TypedColumn[T, A] = frameless.functions.lit(value) /** Aggregate function: returns the number of items in a group. * * apache/spark */ - def count(): TypedAggregate[Long] = { + def count[T](): TypedAggregate[T, Long] = { new TypedAggregate(untyped.count(untyped.lit(1))) } @@ -26,22 +26,22 @@ trait AggregateFunctions { * * apache/spark */ - def count(column: TypedColumn[_]): TypedAggregate[Long] = { - new TypedAggregate[Long](untyped.count(column.untyped)) + def count[T](column: TypedColumn[T, _]): TypedAggregate[T, Long] = { + new TypedAggregate[T, Long](untyped.count(column.untyped)) } /** Aggregate function: returns the number of distinct items in a group. * * apache/spark */ - def countDistinct(column: TypedColumn[_]): TypedAggregate[Long] = { - new TypedAggregate[Long](untyped.countDistinct(column.untyped)) + def countDistinct[T](column: TypedColumn[T, _]): TypedAggregate[T, Long] = { + new TypedAggregate[T, Long](untyped.countDistinct(column.untyped)) } /** Aggregate function: returns the approximate number of distinct items in a group. */ - def approxCountDistinct(column: TypedColumn[_]): TypedAggregate[Long] = { - new TypedAggregate[Long](untyped.approx_count_distinct(column.untyped)) + def approxCountDistinct[T](column: TypedColumn[T, _]): TypedAggregate[T, Long] = { + new TypedAggregate[T, Long](untyped.approx_count_distinct(column.untyped)) } /** Aggregate function: returns the approximate number of distinct items in a group. @@ -50,68 +50,68 @@ trait AggregateFunctions { * * apache/spark */ - def approxCountDistinct(column: TypedColumn[_], rsd: Double): TypedAggregate[Long] = { - new TypedAggregate[Long](untyped.approx_count_distinct(column.untyped, rsd)) + def approxCountDistinct[T](column: TypedColumn[T, _], rsd: Double): TypedAggregate[T, Long] = { + new TypedAggregate[T, Long](untyped.approx_count_distinct(column.untyped, rsd)) } /** Aggregate function: returns a list of objects with duplicates. * * apache/spark */ - def collectList[A: TypedEncoder](column: TypedColumn[A]): TypedAggregate[Vector[A]] = { - new TypedAggregate[Vector[A]](untyped.collect_list(column.untyped)) + def collectList[T, A: TypedEncoder](column: TypedColumn[T, A]): TypedAggregate[T, Vector[A]] = { + new TypedAggregate[T, Vector[A]](untyped.collect_list(column.untyped)) } /** Aggregate function: returns a set of objects with duplicate elements eliminated. * * apache/spark */ - def collectSet[A: TypedEncoder](column: TypedColumn[A]): TypedAggregate[Vector[A]] = { - new TypedAggregate[Vector[A]](untyped.collect_set(column.untyped)) + def collectSet[T, A: TypedEncoder](column: TypedColumn[T, A]): TypedAggregate[T, Vector[A]] = { + new TypedAggregate[T, Vector[A]](untyped.collect_set(column.untyped)) } /** Aggregate function: returns the sum of all values in the given column. * * apache/spark */ - def sum[A, Out](column: TypedColumn[A])( + def sum[A, T, Out](column: TypedColumn[T, A])( implicit summable: CatalystSummable[A, Out], oencoder: TypedEncoder[Out] - ): TypedAggregate[Out] = { + ): TypedAggregate[T, Out] = { val zeroExpr = Literal.create(summable.zero, TypedEncoder[Out].targetDataType) val sumExpr = expr(untyped.sum(column.untyped)) val sumOrZero = Coalesce(Seq(sumExpr, zeroExpr)) - new TypedAggregate[Out](sumOrZero) + new TypedAggregate[T, Out](sumOrZero) } /** Aggregate function: returns the sum of distinct values in the column. * * apache/spark */ - def sumDistinct[A, Out](column: TypedColumn[A])( + def sumDistinct[A, T, Out](column: TypedColumn[T, A])( implicit summable: CatalystSummable[A, Out], oencoder: TypedEncoder[Out] - ): TypedAggregate[Out] = { + ): TypedAggregate[T, Out] = { val zeroExpr = Literal.create(summable.zero, TypedEncoder[Out].targetDataType) val sumExpr = expr(untyped.sumDistinct(column.untyped)) val sumOrZero = Coalesce(Seq(sumExpr, zeroExpr)) - new TypedAggregate[Out](sumOrZero) + new TypedAggregate[T, Out](sumOrZero) } /** Aggregate function: returns the average of the values in a group. * * apache/spark */ - def avg[A, Out](column: TypedColumn[A])( + def avg[A, T, Out](column: TypedColumn[T, A])( implicit averageable: CatalystAverageable[A, Out], oencoder: TypedEncoder[Out] - ): TypedAggregate[Out] = { - new TypedAggregate[Out](untyped.avg(column.untyped)) + ): TypedAggregate[T, Out] = { + new TypedAggregate[T, Out](untyped.avg(column.untyped)) } @@ -122,8 +122,8 @@ trait AggregateFunctions { * * apache/spark */ - def variance[A: CatalystVariance](column: TypedColumn[A]): TypedAggregate[Double] = { - new TypedAggregate[Double](untyped.variance(column.untyped)) + def variance[A: CatalystVariance, T](column: TypedColumn[T, A]): TypedAggregate[T, Double] = { + new TypedAggregate[T, Double](untyped.variance(column.untyped)) } /** Aggregate function: returns the sample standard deviation. @@ -133,8 +133,8 @@ trait AggregateFunctions { * * apache/spark */ - def stddev[A: CatalystVariance](column: TypedColumn[A]): TypedAggregate[Double] = { - new TypedAggregate[Double](untyped.stddev(column.untyped)) + def stddev[A: CatalystVariance, T](column: TypedColumn[T, A]): TypedAggregate[T, Double] = { + new TypedAggregate[T, Double](untyped.stddev(column.untyped)) } /** @@ -145,13 +145,13 @@ trait AggregateFunctions { * * apache/spark */ - def stddevPop[A](column: TypedColumn[A])( + def stddevPop[A, T](column: TypedColumn[T, A])( implicit evCanBeDoubleA: CatalystCast[A, Double] - ): TypedAggregate[Option[Double]] = { + ): TypedAggregate[T, Option[Double]] = { implicit val c1 = column.uencoder - new TypedAggregate[Option[Double]]( + new TypedAggregate[T, Option[Double]]( untyped.stddev_pop(column.cast[Double].untyped) ) } @@ -164,13 +164,13 @@ trait AggregateFunctions { * * apache/spark */ - def stddevSamp[A](column: TypedColumn[A])( + def stddevSamp[A, T](column: TypedColumn[T, A])( implicit evCanBeDoubleA: CatalystCast[A, Double] - ): TypedAggregate[Option[Double]] = { + ): TypedAggregate[T, Option[Double]] = { implicit val c1 = column.uencoder - new TypedAggregate[Option[Double]]( + new TypedAggregate[T, Option[Double]]( untyped.stddev_samp(column.cast[Double].untyped) ) } @@ -179,18 +179,18 @@ trait AggregateFunctions { * * apache/spark */ - def max[A: CatalystOrdered](column: TypedColumn[A]): TypedAggregate[A] = { + def max[A: CatalystOrdered, T](column: TypedColumn[T, A]): TypedAggregate[T, A] = { implicit val c = column.uencoder - new TypedAggregate[A](untyped.max(column.untyped)) + new TypedAggregate[T, A](untyped.max(column.untyped)) } /** Aggregate function: returns the minimum value of the column in a group. * * apache/spark */ - def min[A: CatalystOrdered](column: TypedColumn[A]): TypedAggregate[A] = { + def min[A: CatalystOrdered, T](column: TypedColumn[T, A]): TypedAggregate[T, A] = { implicit val c = column.uencoder - new TypedAggregate[A](untyped.min(column.untyped)) + new TypedAggregate[T, A](untyped.min(column.untyped)) } /** Aggregate function: returns the first value in a group. @@ -200,9 +200,9 @@ trait AggregateFunctions { * * apache/spark */ - def first[A](column: TypedColumn[A]): TypedAggregate[A] = { + def first[A, T](column: TypedColumn[T, A]): TypedAggregate[T, A] = { implicit val c = column.uencoder - new TypedAggregate[A](untyped.first(column.untyped)) + new TypedAggregate[T, A](untyped.first(column.untyped)) } /** @@ -213,9 +213,9 @@ trait AggregateFunctions { * * apache/spark */ - def last[A](column: TypedColumn[A]): TypedAggregate[A] = { + def last[A, T](column: TypedColumn[T, A]): TypedAggregate[T, A] = { implicit val c = column.uencoder - new TypedAggregate[A](untyped.last(column.untyped)) + new TypedAggregate[T, A](untyped.last(column.untyped)) } /** @@ -226,15 +226,15 @@ trait AggregateFunctions { * * apache/spark */ - def corr[A, B](column1: TypedColumn[A], column2: TypedColumn[B])( + def corr[A, B, T](column1: TypedColumn[T, A], column2: TypedColumn[T, B])( implicit evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] - ): TypedAggregate[Option[Double]] = { + ): TypedAggregate[T, Option[Double]] = { implicit val c1 = column1.uencoder implicit val c2 = column2.uencoder - new TypedAggregate[Option[Double]]( + new TypedAggregate[T, Option[Double]]( untyped.corr(column1.cast[Double].untyped, column2.cast[Double].untyped) ) } @@ -247,15 +247,15 @@ trait AggregateFunctions { * * apache/spark */ - def covarPop[A, B](column1: TypedColumn[A], column2: TypedColumn[B])( + def covarPop[A, B, T](column1: TypedColumn[T, A], column2: TypedColumn[T, B])( implicit evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] - ): TypedAggregate[Option[Double]] = { + ): TypedAggregate[T, Option[Double]] = { implicit val c1 = column1.uencoder implicit val c2 = column2.uencoder - new TypedAggregate[Option[Double]]( + new TypedAggregate[T, Option[Double]]( untyped.covar_pop(column1.cast[Double].untyped, column2.cast[Double].untyped) ) } @@ -268,15 +268,15 @@ trait AggregateFunctions { * * apache/spark */ - def covarSamp[A, B](column1: TypedColumn[A], column2: TypedColumn[B])( + def covarSamp[A, B, T](column1: TypedColumn[T, A], column2: TypedColumn[T, B])( implicit evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] - ): TypedAggregate[Option[Double]] = { + ): TypedAggregate[T, Option[Double]] = { implicit val c1 = column1.uencoder implicit val c2 = column2.uencoder - new TypedAggregate[Option[Double]]( + new TypedAggregate[T, Option[Double]]( untyped.covar_samp(column1.cast[Double].untyped, column2.cast[Double].untyped) ) } @@ -290,13 +290,13 @@ trait AggregateFunctions { * * apache/spark */ - def kurtosis[A](column: TypedColumn[A])( + def kurtosis[A, T](column: TypedColumn[T, A])( implicit evCanBeDoubleA: CatalystCast[A, Double] - ): TypedAggregate[Option[Double]] = { + ): TypedAggregate[T, Option[Double]] = { implicit val c1 = column.uencoder - new TypedAggregate[Option[Double]]( + new TypedAggregate[T, Option[Double]]( untyped.kurtosis(column.cast[Double].untyped) ) } @@ -309,14 +309,14 @@ trait AggregateFunctions { * * apache/spark */ - def skewness[A](column: TypedColumn[A])( + def skewness[A, T](column: TypedColumn[T, A])( implicit evCanBeDoubleA: CatalystCast[A, Double] - ): TypedAggregate[Option[Double]] = { + ): TypedAggregate[T, Option[Double]] = { implicit val c1 = column.uencoder - new TypedAggregate[Option[Double]]( + new TypedAggregate[T, Option[Double]]( untyped.skewness(column.cast[Double].untyped) ) } -} +} \ No newline at end of file diff --git a/dataset/src/main/scala/frameless/functions/Udf.scala b/dataset/src/main/scala/frameless/functions/Udf.scala index 6e1256eab..0b74d7a47 100644 --- a/dataset/src/main/scala/frameless/functions/Udf.scala +++ b/dataset/src/main/scala/frameless/functions/Udf.scala @@ -20,10 +20,10 @@ trait Udf { * apache/spark */ def udf[T, A, R: TypedEncoder](f: A => R): - TypedColumn[A] => TypedColumn[R] = { + TypedColumn[T, A] => TypedColumn[T, R] = { u => val scalaUdf = FramelessUdf(f, List(u), TypedEncoder[R]) - new TypedColumn[R](scalaUdf) + new TypedColumn[T, R](scalaUdf) } /** Defines a user-defined function of 2 arguments as user-defined function (UDF). @@ -32,10 +32,10 @@ trait Udf { * apache/spark */ def udf[T, A1, A2, R: TypedEncoder](f: (A1,A2) => R): - (TypedColumn[A1], TypedColumn[A2]) => TypedColumn[R] = { + (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R] = { case us => - val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression], TypedEncoder[R]) - new TypedColumn[R](scalaUdf) + val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression[T]], TypedEncoder[R]) + new TypedColumn[T, R](scalaUdf) } /** Defines a user-defined function of 3 arguments as user-defined function (UDF). @@ -44,10 +44,10 @@ trait Udf { * apache/spark */ def udf[T, A1, A2, A3, R: TypedEncoder](f: (A1,A2,A3) => R): - (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3]) => TypedColumn[R] = { + (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3]) => TypedColumn[T, R] = { case us => - val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression], TypedEncoder[R]) - new TypedColumn[R](scalaUdf) + val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression[T]], TypedEncoder[R]) + new TypedColumn[T, R](scalaUdf) } /** Defines a user-defined function of 4 arguments as user-defined function (UDF). @@ -56,10 +56,10 @@ trait Udf { * apache/spark */ def udf[T, A1, A2, A3, A4, R: TypedEncoder](f: (A1,A2,A3,A4) => R): - (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4]) => TypedColumn[R] = { + (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4]) => TypedColumn[T, R] = { case us => - val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression], TypedEncoder[R]) - new TypedColumn[R](scalaUdf) + val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression[T]], TypedEncoder[R]) + new TypedColumn[T, R](scalaUdf) } /** Defines a user-defined function of 5 arguments as user-defined function (UDF). @@ -68,10 +68,10 @@ trait Udf { * apache/spark */ def udf[T, A1, A2, A3, A4, A5, R: TypedEncoder](f: (A1,A2,A3,A4,A5) => R): - (TypedColumn[A1], TypedColumn[A2], TypedColumn[A3], TypedColumn[A4], TypedColumn[A5]) => TypedColumn[R] = { + (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4], TypedColumn[T, A5]) => TypedColumn[T, R] = { case us => - val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression], TypedEncoder[R]) - new TypedColumn[R](scalaUdf) + val scalaUdf = FramelessUdf(f, us.toList[UntypedExpression[T]], TypedEncoder[R]) + new TypedColumn[T, R](scalaUdf) } } @@ -178,7 +178,7 @@ object FramelessUdf { // Spark needs case class with `children` field to mutate it def apply[T, R]( function: AnyRef, - cols: Seq[UntypedExpression], + cols: Seq[UntypedExpression[T]], rencoder: TypedEncoder[R] ): FramelessUdf[T, R] = FramelessUdf( function = function, diff --git a/dataset/src/main/scala/frameless/functions/UnaryFunctions.scala b/dataset/src/main/scala/frameless/functions/UnaryFunctions.scala index e8e407ee8..efce32925 100644 --- a/dataset/src/main/scala/frameless/functions/UnaryFunctions.scala +++ b/dataset/src/main/scala/frameless/functions/UnaryFunctions.scala @@ -10,24 +10,24 @@ trait UnaryFunctions { * * apache/spark */ - def size[T, A, V[_] : CatalystSizableCollection](column: TypedColumn[V[A]]): TypedColumn[Int] = - new TypedColumn[Int](implicitly[CatalystSizableCollection[V]].sizeOp(column.untyped)) + def size[T, A, V[_] : CatalystSizableCollection](column: TypedColumn[T, V[A]]): TypedColumn[T, Int] = + new TypedColumn[T, Int](implicitly[CatalystSizableCollection[V]].sizeOp(column.untyped)) /** Sorts the input array for the given column in ascending order, according to * the natural ordering of the array elements. * * apache/spark */ - def sortAscending[T, A: Ordering, V[_] : CatalystSortableCollection](column: TypedColumn[V[A]]): TypedColumn[V[A]] = - new TypedColumn[V[A]](implicitly[CatalystSortableCollection[V]].sortOp(column.untyped, sortAscending = true))(column.uencoder) + def sortAscending[T, A: Ordering, V[_] : CatalystSortableCollection](column: TypedColumn[T, V[A]]): TypedColumn[T, V[A]] = + new TypedColumn[T, V[A]](implicitly[CatalystSortableCollection[V]].sortOp(column.untyped, sortAscending = true))(column.uencoder) /** Sorts the input array for the given column in descending order, according to * the natural ordering of the array elements. * * apache/spark */ - def sortDescending[T, A: Ordering, V[_] : CatalystSortableCollection](column: TypedColumn[V[A]]): TypedColumn[V[A]] = - new TypedColumn[V[A]](implicitly[CatalystSortableCollection[V]].sortOp(column.untyped, sortAscending = false))(column.uencoder) + def sortDescending[T, A: Ordering, V[_] : CatalystSortableCollection](column: TypedColumn[T, V[A]]): TypedColumn[T, V[A]] = + new TypedColumn[T, V[A]](implicitly[CatalystSortableCollection[V]].sortOp(column.untyped, sortAscending = false))(column.uencoder) /** Creates a new row for each element in the given collection. The column types @@ -35,8 +35,8 @@ trait UnaryFunctions { * * apache/spark */ - def explode[T, A: TypedEncoder, V[_] : CatalystExplodableCollection](column: TypedColumn[V[A]]): TypedColumn[A] = - new TypedColumn[A](sparkFunctions.explode(column.untyped)) + def explode[T, A: TypedEncoder, V[_] : CatalystExplodableCollection](column: TypedColumn[T, V[A]]): TypedColumn[T, A] = + new TypedColumn[T, A](sparkFunctions.explode(column.untyped)) } trait CatalystSizableCollection[V[_]] { @@ -63,4 +63,4 @@ object CatalystSortableCollection { implicit def sortableVector: CatalystSortableCollection[Vector] = new CatalystSortableCollection[Vector] { def sortOp(col: Column, sortAscending: Boolean): Column = sparkFunctions.sort_array(col, sortAscending) } -} +} \ No newline at end of file diff --git a/dataset/src/main/scala/frameless/functions/package.scala b/dataset/src/main/scala/frameless/functions/package.scala index 51350621b..ccecdb495 100644 --- a/dataset/src/main/scala/frameless/functions/package.scala +++ b/dataset/src/main/scala/frameless/functions/package.scala @@ -6,7 +6,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal package object functions extends Udf with UnaryFunctions { object aggregate extends AggregateFunctions - def lit[A: TypedEncoder, T](value: A): TypedColumn[A] = { + def lit[A: TypedEncoder, T](value: A): TypedColumn[T, A] = { val encoder = TypedEncoder[A] if (ScalaReflection.isNativeType(encoder.sourceDataType) && encoder.targetDataType == encoder.sourceDataType) { diff --git a/dataset/src/main/scala/frameless/implicits.scala b/dataset/src/main/scala/frameless/implicits.scala index 32ca3a5ee..f70c5f042 100644 --- a/dataset/src/main/scala/frameless/implicits.scala +++ b/dataset/src/main/scala/frameless/implicits.scala @@ -4,32 +4,32 @@ object implicits { object widen { // frameless prefixed to avoid implicit name collision - implicit def framelessByteToShort[T](col: TypedColumn[Byte]): TypedColumn[Short] = col.cast[Short] - implicit def framelessByteToInt[T](col: TypedColumn[Byte]): TypedColumn[Int] = col.cast[Int] - implicit def framelessByteToLong[T](col: TypedColumn[Byte]): TypedColumn[Long] = col.cast[Long] - implicit def framelessByteToDouble[T](col: TypedColumn[Byte]): TypedColumn[Double] = col.cast[Double] - implicit def framelessByteToBigDecimal[T](col: TypedColumn[Byte]): TypedColumn[BigDecimal] = col.cast[BigDecimal] + implicit def framelessByteToShort[T](col: TypedColumn[T, Byte]): TypedColumn[T, Short] = col.cast[Short] + implicit def framelessByteToInt[T](col: TypedColumn[T, Byte]): TypedColumn[T, Int] = col.cast[Int] + implicit def framelessByteToLong[T](col: TypedColumn[T, Byte]): TypedColumn[T, Long] = col.cast[Long] + implicit def framelessByteToDouble[T](col: TypedColumn[T, Byte]): TypedColumn[T, Double] = col.cast[Double] + implicit def framelessByteToBigDecimal[T](col: TypedColumn[T, Byte]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] - implicit def framelessShortToInt[T](col: TypedColumn[Short]): TypedColumn[Int] = col.cast[Int] - implicit def framelessShortToLong[T](col: TypedColumn[Short]): TypedColumn[Long] = col.cast[Long] - implicit def framelessShortToDouble[T](col: TypedColumn[Short]): TypedColumn[Double] = col.cast[Double] - implicit def framelessShortToBigDecimal[T](col: TypedColumn[Short]): TypedColumn[BigDecimal] = col.cast[BigDecimal] + implicit def framelessShortToInt[T](col: TypedColumn[T, Short]): TypedColumn[T, Int] = col.cast[Int] + implicit def framelessShortToLong[T](col: TypedColumn[T, Short]): TypedColumn[T, Long] = col.cast[Long] + implicit def framelessShortToDouble[T](col: TypedColumn[T, Short]): TypedColumn[T, Double] = col.cast[Double] + implicit def framelessShortToBigDecimal[T](col: TypedColumn[T, Short]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] - implicit def framelessIntToLong[T](col: TypedColumn[Int]): TypedColumn[Long] = col.cast[Long] - implicit def framelessIntToDouble[T](col: TypedColumn[Int]): TypedColumn[Double] = col.cast[Double] - implicit def framelessIntToBigDecimal[T](col: TypedColumn[Int]): TypedColumn[BigDecimal] = col.cast[BigDecimal] + implicit def framelessIntToLong[T](col: TypedColumn[T, Int]): TypedColumn[T, Long] = col.cast[Long] + implicit def framelessIntToDouble[T](col: TypedColumn[T, Int]): TypedColumn[T, Double] = col.cast[Double] + implicit def framelessIntToBigDecimal[T](col: TypedColumn[T, Int]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] - implicit def framelessLongToDouble[T](col: TypedColumn[Long]): TypedColumn[Double] = col.cast[Double] - implicit def framelessLongToBigDecimal[T](col: TypedColumn[Long]): TypedColumn[BigDecimal] = col.cast[BigDecimal] + implicit def framelessLongToDouble[T](col: TypedColumn[T, Long]): TypedColumn[T, Double] = col.cast[Double] + implicit def framelessLongToBigDecimal[T](col: TypedColumn[T, Long]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] - implicit def framelessDoubleToBigDecimal[T](col: TypedColumn[Double]): TypedColumn[BigDecimal] = col.cast[BigDecimal] + implicit def framelessDoubleToBigDecimal[T](col: TypedColumn[T, Double]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] // we don't have floats yet, but then this is lawful (or not?): // - // implicit def byteToFloat[T](col: TypedColumn[Byte]): TypedColumn[Float] = col.cast[Float] - // implicit def intToFloat[T](col: TypedColumn[Int]): TypedColumn[Float] = col.cast[Float] - // implicit def longToFloat[T](col: TypedColumn[Long]): TypedColumn[Float] = col.cast[Float] - // implicit def floatToDouble[T](col: TypedColumn[Float]): TypedColumn[Double] = col.cast[Double] - // implicit def floatToBigDecimal[T](col: TypedColumn[Float]): TypedColumn[BigDecimal] = col.cast[BigDecimal] + // implicit def byteToFloat[T](col: TypedColumn[T, Byte]): TypedColumn[T, Float] = col.cast[Float] + // implicit def intToFloat[T](col: TypedColumn[T, Int]): TypedColumn[T, Float] = col.cast[Float] + // implicit def longToFloat[T](col: TypedColumn[T, Long]): TypedColumn[T, Float] = col.cast[Float] + // implicit def floatToDouble[T](col: TypedColumn[T, Float]): TypedColumn[T, Double] = col.cast[Double] + // implicit def floatToBigDecimal[T](col: TypedColumn[T, Float]): TypedColumn[T, BigDecimal] = col.cast[BigDecimal] } } diff --git a/dataset/src/main/scala/frameless/ops/AggregateTypes.scala b/dataset/src/main/scala/frameless/ops/AggregateTypes.scala index bc38be616..403c25301 100644 --- a/dataset/src/main/scala/frameless/ops/AggregateTypes.scala +++ b/dataset/src/main/scala/frameless/ops/AggregateTypes.scala @@ -23,6 +23,6 @@ object AggregateTypes { implicit def deriveCons1[T, H, TT <: HList, V <: HList]( implicit tail: AggregateTypes.Aux[T, TT, V] - ): AggregateTypes.Aux[T, TypedAggregate[H] :: TT, H :: V] = - new AggregateTypes[T, TypedAggregate[H] :: TT] {type Out = H :: V} + ): AggregateTypes.Aux[T, TypedAggregate[T, H] :: TT, H :: V] = + new AggregateTypes[T, TypedAggregate[T, H] :: TT] {type Out = H :: V} } diff --git a/dataset/src/main/scala/frameless/ops/ColumnTypes.scala b/dataset/src/main/scala/frameless/ops/ColumnTypes.scala index b0606febe..e5ae6aea2 100644 --- a/dataset/src/main/scala/frameless/ops/ColumnTypes.scala +++ b/dataset/src/main/scala/frameless/ops/ColumnTypes.scala @@ -8,21 +8,21 @@ import shapeless._ * @note This type class is mostly a workaround to issue with slow implicit derivation for Comapped. * @example * {{{ - * type U = TypedColumn[A] :: TypedColumn[B] :: TypedColumn[C] :: HNil + * type U = TypedColumn[T,A] :: TypedColumn[T,B] :: TypedColumn[T,C] :: HNil * type Out = A :: B :: C :: HNil * }}} */ -trait ColumnTypes[U <: HList] { +trait ColumnTypes[T, U <: HList] { type Out <: HList } object ColumnTypes { - type Aux[U <: HList, Out0 <: HList] = ColumnTypes[U] {type Out = Out0} + type Aux[T, U <: HList, Out0 <: HList] = ColumnTypes[T, U] {type Out = Out0} - implicit def deriveHNil: ColumnTypes.Aux[HNil, HNil] = new ColumnTypes[HNil] { type Out = HNil } + implicit def deriveHNil[T]: ColumnTypes.Aux[T, HNil, HNil] = new ColumnTypes[T, HNil] { type Out = HNil } - implicit def deriveCons[H, TT <: HList, V <: HList]( - implicit tail: ColumnTypes.Aux[TT, V] - ): ColumnTypes.Aux[TypedColumn[H] :: TT, H :: V] = - new ColumnTypes[TypedColumn[H] :: TT] {type Out = H :: V} + implicit def deriveCons[T, H, TT <: HList, V <: HList]( + implicit tail: ColumnTypes.Aux[T, TT, V] + ): ColumnTypes.Aux[T, TypedColumn[T, H] :: TT, H :: V] = + new ColumnTypes[T, TypedColumn[T, H] :: TT] {type Out = H :: V} } diff --git a/dataset/src/main/scala/frameless/ops/GroupByOps.scala b/dataset/src/main/scala/frameless/ops/GroupByOps.scala index 1aa7b691e..a217a3216 100644 --- a/dataset/src/main/scala/frameless/ops/GroupByOps.scala +++ b/dataset/src/main/scala/frameless/ops/GroupByOps.scala @@ -12,8 +12,8 @@ class GroupedByManyOps[T, TK <: HList, K <: HList, KT]( groupedBy: TK )( implicit - ct: ColumnTypes.Aux[TK, K], - toTraversable: ToTraversable.Aux[TK, List, UntypedExpression], + ct: ColumnTypes.Aux[T, TK, K], + toTraversable: ToTraversable.Aux[TK, List, UntypedExpression[T]], tupler: Tupler.Aux[K, KT] ) { @@ -24,10 +24,10 @@ class GroupedByManyOps[T, TK <: HList, K <: HList, KT]( append: Prepend.Aux[K, C, Out0], toTuple: Tupler.Aux[Out0, Out1], encoder: TypedEncoder[Out1], - columnsToList: ToTraversable.Aux[TC, List, UntypedExpression] + columnsToList: ToTraversable.Aux[TC, List, UntypedExpression[T]] ): TypedDataset[Out1] = { - def expr(c: UntypedExpression): Column = new Column(c.expr) + def expr(c: UntypedExpression[T]): Column = new Column(c.expr) val groupByExprs = toTraversable(groupedBy).map(expr) val aggregates = @@ -93,32 +93,32 @@ object GroupedByManyOps { class GroupedBy1Ops[K1, V]( self: TypedDataset[V], - g1: TypedColumn[K1] + g1: TypedColumn[V, K1] ) { private def underlying = new GroupedByManyOps(self, g1 :: HNil) private implicit def eg1 = g1.uencoder - def agg[U1](c1: TypedAggregate[U1]): TypedDataset[(K1, U1)] = { + def agg[U1](c1: TypedAggregate[V, U1]): TypedDataset[(K1, U1)] = { implicit val e1 = c1.uencoder underlying.agg(c1) } - def agg[U1, U2](c1: TypedAggregate[U1], c2: TypedAggregate[U2]): TypedDataset[(K1, U1, U2)] = { + def agg[U1, U2](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2]): TypedDataset[(K1, U1, U2)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder underlying.agg(c1, c2) } - def agg[U1, U2, U3](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3]): TypedDataset[(K1, U1, U2, U3)] = { + def agg[U1, U2, U3](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3]): TypedDataset[(K1, U1, U2, U3)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder underlying.agg(c1, c2, c3) } - def agg[U1, U2, U3, U4](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3], c4: TypedAggregate[U4]): TypedDataset[(K1, U1, U2, U3, U4)] = { + def agg[U1, U2, U3, U4](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3], c4: TypedAggregate[V, U4]): TypedDataset[(K1, U1, U2, U3, U4)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder; implicit val e4 = c4.uencoder underlying.agg(c1, c2, c3, c4) } - def agg[U1, U2, U3, U4, U5](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3], c4: TypedAggregate[U4], c5: TypedAggregate[U5]): TypedDataset[(K1, U1, U2, U3, U4, U5)] = { + def agg[U1, U2, U3, U4, U5](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3], c4: TypedAggregate[V, U4], c5: TypedAggregate[V, U5]): TypedDataset[(K1, U1, U2, U3, U4, U5)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder; implicit val e4 = c4.uencoder; implicit val e5 = c5.uencoder underlying.agg(c1, c2, c3, c4, c5) } @@ -134,34 +134,34 @@ class GroupedBy1Ops[K1, V]( class GroupedBy2Ops[K1, K2, V]( self: TypedDataset[V], - g1: TypedColumn[K1], - g2: TypedColumn[K2] + g1: TypedColumn[V, K1], + g2: TypedColumn[V, K2] ) { private def underlying = new GroupedByManyOps(self, g1 :: g2 :: HNil) private implicit def eg1 = g1.uencoder private implicit def eg2 = g2.uencoder - def agg[U1](c1: TypedAggregate[U1]): TypedDataset[(K1, K2, U1)] = { + def agg[U1](c1: TypedAggregate[V, U1]): TypedDataset[(K1, K2, U1)] = { implicit val e1 = c1.uencoder underlying.agg(c1) } - def agg[U1, U2](c1: TypedAggregate[U1], c2: TypedAggregate[U2]): TypedDataset[(K1, K2, U1, U2)] = { + def agg[U1, U2](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2]): TypedDataset[(K1, K2, U1, U2)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder underlying.agg(c1, c2) } - def agg[U1, U2, U3](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3]): TypedDataset[(K1, K2, U1, U2, U3)] = { + def agg[U1, U2, U3](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3]): TypedDataset[(K1, K2, U1, U2, U3)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder underlying.agg(c1, c2, c3) } - def agg[U1, U2, U3, U4](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3], c4: TypedAggregate[U4]): TypedDataset[(K1, K2, U1, U2, U3, U4)] = { + def agg[U1, U2, U3, U4](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3], c4: TypedAggregate[V, U4]): TypedDataset[(K1, K2, U1, U2, U3, U4)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder; implicit val e4 = c4.uencoder underlying.agg(c1 , c2 , c3 , c4) } - def agg[U1, U2, U3, U4, U5](c1: TypedAggregate[U1], c2: TypedAggregate[U2], c3: TypedAggregate[U3], c4: TypedAggregate[U4], c5: TypedAggregate[U5]): TypedDataset[(K1, K2, U1, U2, U3, U4, U5)] = { + def agg[U1, U2, U3, U4, U5](c1: TypedAggregate[V, U1], c2: TypedAggregate[V, U2], c3: TypedAggregate[V, U3], c4: TypedAggregate[V, U4], c5: TypedAggregate[V, U5]): TypedDataset[(K1, K2, U1, U2, U3, U4, U5)] = { implicit val e1 = c1.uencoder; implicit val e2 = c2.uencoder; implicit val e3 = c3.uencoder; implicit val e4 = c4.uencoder; implicit val e5 = c5.uencoder underlying.agg(c1, c2, c3, c4, c5) } diff --git a/dataset/src/test/scala/frameless/ColTests.scala b/dataset/src/test/scala/frameless/ColTests.scala index 35aeae8c7..ad62aa068 100644 --- a/dataset/src/test/scala/frameless/ColTests.scala +++ b/dataset/src/test/scala/frameless/ColTests.scala @@ -33,10 +33,10 @@ class ColTests extends TypedDatasetSuite { type X2X2 = X2[X2[Int, String], X2[Long, Boolean]] val x2x2 = TypedDataset.create[X2X2](Nil) - val aa: TypedColumn[Int] = x2x2.colMany('a, 'a) - val ab: TypedColumn[String] = x2x2.colMany('a, 'b) - val ba: TypedColumn[Long] = x2x2.colMany('b, 'a) - val bb: TypedColumn[Boolean] = x2x2.colMany('b, 'b) + val aa: TypedColumn[X2X2, Int] = x2x2.colMany('a, 'a) + val ab: TypedColumn[X2X2, String] = x2x2.colMany('a, 'b) + val ba: TypedColumn[X2X2, Long] = x2x2.colMany('b, 'a) + val bb: TypedColumn[X2X2, Boolean] = x2x2.colMany('b, 'b) illTyped("x2x2.colMany('a, 'c)") illTyped("x2x2.colMany('a, 'a, 'a)") diff --git a/dataset/src/test/scala/frameless/SelectTests.scala b/dataset/src/test/scala/frameless/SelectTests.scala index 48c92f231..b6b9772c8 100644 --- a/dataset/src/test/scala/frameless/SelectTests.scala +++ b/dataset/src/test/scala/frameless/SelectTests.scala @@ -397,4 +397,4 @@ class SelectTests extends TypedDatasetSuite { val e = TypedDataset.create[(Int, String, Long)]((1, "a", 2L) :: (2, "b", 4L) :: (2, "b", 1L) :: Nil) illTyped("""e.select(frameless.functions.aggregate.sum(e('_1)))""") } -} +} \ No newline at end of file diff --git a/dataset/src/test/scala/frameless/WidenTests.scala b/dataset/src/test/scala/frameless/WidenTests.scala index ee486aa14..7b2eaa2fe 100644 --- a/dataset/src/test/scala/frameless/WidenTests.scala +++ b/dataset/src/test/scala/frameless/WidenTests.scala @@ -16,7 +16,7 @@ class WidenTests extends TypedDatasetSuite { def widenSum[A: TypedEncoder: CatalystNumeric: Numeric, B: TypedEncoder](a: A, b: B)( implicit view: B => A, - colView: TypedColumn[B] => TypedColumn[A] + colView: TypedColumn[X2[A, B], B] => TypedColumn[X2[A, B], A] ): Prop = { val df = TypedDataset.create(X2(a, b) :: Nil) val sum = implicitly[Numeric[A]].plus(a, view(b)) @@ -31,7 +31,7 @@ class WidenTests extends TypedDatasetSuite { def widen[A: TypedEncoder, B: TypedEncoder](a: A)( implicit view: A => B, - colView: TypedColumn[A] => TypedColumn[B] + colView: TypedColumn[X1[A], A] => TypedColumn[X1[A], B] ): Prop = { val df = TypedDataset.create(X1(a) :: Nil) val got = df.select(colView(df.col('a))).collect().run() diff --git a/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala b/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala index a70a3066b..99729cd64 100644 --- a/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala +++ b/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala @@ -23,7 +23,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { else falsified :| s"Expected $a but got $b, which is more than 1% off and greater than epsilon = $epsilon." } - def sparkSchema[A: TypedEncoder, U](f: TypedColumn[A] => TypedAggregate[U]): Prop = { + def sparkSchema[A: TypedEncoder, U](f: TypedColumn[X1[A], A] => TypedAggregate[X1[A], U]): Prop = { val df = TypedDataset.create[X1[A]](Nil) val col = f(df.col('a)) @@ -366,7 +366,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { xs: List[X3[Int, A, B]] ) ( - framelessFun: (TypedColumn[A], TypedColumn[B]) => TypedAggregate[Option[Double]], + framelessFun: (TypedColumn[X3[Int, A, B], A], TypedColumn[X3[Int, A, B], B]) => TypedAggregate[X3[Int, A, B], Option[Double]], sparkFun: (Column, Column) => Column ) ( @@ -405,7 +405,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { xs: List[X2[Int, A]] ) ( - framelessFun: (TypedColumn[A]) => TypedAggregate[Option[Double]], + framelessFun: (TypedColumn[X2[Int, A], A]) => TypedAggregate[X2[Int, A], Option[Double]], sparkFun: (Column) => Column ) ( @@ -447,7 +447,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { encEv: Encoder[(Int, A, B)], evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] - ): Prop = bivariatePropTemplate(xs)(corr[A, B],org.apache.spark.sql.functions.corr) + ): Prop = bivariatePropTemplate(xs)(corr[A,B,X3[Int, A, B]],org.apache.spark.sql.functions.corr) check(forAll(prop[Double, Double] _)) check(forAll(prop[Double, Int] _)) @@ -466,7 +466,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] ): Prop = bivariatePropTemplate(xs)( - covarPop[A, B], + covarPop[A, B, X3[Int, A, B]], org.apache.spark.sql.functions.covar_pop ) @@ -487,7 +487,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { evCanBeDoubleA: CatalystCast[A, Double], evCanBeDoubleB: CatalystCast[B, Double] ): Prop = bivariatePropTemplate(xs)( - covarSamp[A, B], + covarSamp[A, B, X3[Int, A, B]], org.apache.spark.sql.functions.covar_samp ) @@ -507,7 +507,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { encEv: Encoder[(Int, A)], evCanBeDoubleA: CatalystCast[A, Double] ): Prop = univariatePropTemplate(xs)( - kurtosis[A], + kurtosis[A, X2[Int, A]], org.apache.spark.sql.functions.kurtosis ) @@ -527,7 +527,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { encEv: Encoder[(Int, A)], evCanBeDoubleA: CatalystCast[A, Double] ): Prop = univariatePropTemplate(xs)( - skewness[A], + skewness[A, X2[Int, A]], org.apache.spark.sql.functions.skewness ) @@ -547,7 +547,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { encEv: Encoder[(Int, A)], evCanBeDoubleA: CatalystCast[A, Double] ): Prop = univariatePropTemplate(xs)( - stddevPop[A], + stddevPop[A, X2[Int, A]], org.apache.spark.sql.functions.stddev_pop ) @@ -567,7 +567,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { encEv: Encoder[(Int, A)], evCanBeDoubleA: CatalystCast[A, Double] ): Prop = univariatePropTemplate(xs)( - stddevSamp[A], + stddevSamp[A, X2[Int, A]], org.apache.spark.sql.functions.stddev_samp ) check(forAll(prop[Double] _)) diff --git a/dataset/src/test/scala/frameless/functions/UnaryFunctionsTest.scala b/dataset/src/test/scala/frameless/functions/UnaryFunctionsTest.scala index 17fe99c4c..d50d6c021 100644 --- a/dataset/src/test/scala/frameless/functions/UnaryFunctionsTest.scala +++ b/dataset/src/test/scala/frameless/functions/UnaryFunctionsTest.scala @@ -69,4 +69,4 @@ class UnaryFunctionsTest extends TypedDatasetSuite { check(forAll(prop[Char] _)) check(forAll(prop[String] _)) } -} +} \ No newline at end of file diff --git a/dataset/src/test/scala/frameless/ops/ColumnTypesTest.scala b/dataset/src/test/scala/frameless/ops/ColumnTypesTest.scala index f40ebf7c6..303eb2cbd 100644 --- a/dataset/src/test/scala/frameless/ops/ColumnTypesTest.scala +++ b/dataset/src/test/scala/frameless/ops/ColumnTypesTest.scala @@ -12,12 +12,12 @@ class ColumnTypesTest extends TypedDatasetSuite { val d: TypedDataset[X4[A, B, C, D]] = TypedDataset.create(data) val hlist = d('a) :: d('b) :: d('c) :: d('d) :: HNil - type TC[N] = TypedColumn[N] + type TC[N] = TypedColumn[X4[A,B,C,D], N] type IN = TC[A] :: TC[B] :: TC[C] :: TC[D] :: HNil type OUT = A :: B :: C :: D :: HNil - implicitly[ColumnTypes.Aux[IN, OUT]] + implicitly[ColumnTypes.Aux[X4[A,B,C,D], IN, OUT]] Prop.passed // successful compilation implies test correctness } From 34b198de2d28816497590de37bd3371b0b822507 Mon Sep 17 00:00:00 2001 From: Olivier Blanvillain Date: Sat, 16 Sep 2017 00:52:57 +0200 Subject: [PATCH 5/6] Remove col type arguments Inference works fine --- .../main/scala/frameless/TypedDataset.scala | 15 ++-- .../src/test/scala/frameless/ColTests.scala | 12 ++-- .../test/scala/frameless/GroupByTests.scala | 66 ++++++++--------- .../test/scala/frameless/SelectTests.scala | 70 +++++++++---------- .../functions/AggregateFunctionsTests.scala | 18 ++--- .../scala/frameless/functions/UdfTests.scala | 40 +++++------ 6 files changed, 113 insertions(+), 108 deletions(-) diff --git a/dataset/src/main/scala/frameless/TypedDataset.scala b/dataset/src/main/scala/frameless/TypedDataset.scala index af01c425b..8bc149f63 100644 --- a/dataset/src/main/scala/frameless/TypedDataset.scala +++ b/dataset/src/main/scala/frameless/TypedDataset.scala @@ -393,31 +393,36 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** Takes a function from A => R and converts it to a UDF for TypedColumn[A] => TypedColumn[R]. */ def makeUDF[A: TypedEncoder, R: TypedEncoder](f: A => R): - TypedColumn[T, A] => TypedColumn[T, R] = functions.udf(f) + 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) + (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) + (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) + (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) + (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] * {{{ diff --git a/dataset/src/test/scala/frameless/ColTests.scala b/dataset/src/test/scala/frameless/ColTests.scala index ad62aa068..59d419fcb 100644 --- a/dataset/src/test/scala/frameless/ColTests.scala +++ b/dataset/src/test/scala/frameless/ColTests.scala @@ -13,18 +13,18 @@ class ColTests extends TypedDatasetSuite { x4.col('a) t4.col('_1) - x4.col[Int]('a) - t4.col[Int]('_1) + x4.col[Int, X4[Int, String, Long, Boolean]]('a) + t4.col[Int, (Int, String, Long, Boolean)]('_1) - illTyped("x4.col[String]('a)", "No column .* of type String in frameless.X4.*") + illTyped("x4.col[String, X4[Int, String, Long, Boolean]]('a)", "No column .* of type String in frameless.X4.*") x4.col('b) t4.col('_2) - x4.col[String]('b) - t4.col[String]('_2) + x4.col[String, X4[Int, String, Long, Boolean]]('b) + t4.col[String, (Int, String, Long, Boolean)]('_2) - illTyped("x4.col[Int]('b)", "No column .* of type Int in frameless.X4.*") + illTyped("x4.col[Int, X4[Int, String, Long, Boolean]]('b)", "No column .* of type Int in frameless.X4.*") () } diff --git a/dataset/src/test/scala/frameless/GroupByTests.scala b/dataset/src/test/scala/frameless/GroupByTests.scala index ea7f3e36d..5828d8cfe 100644 --- a/dataset/src/test/scala/frameless/GroupByTests.scala +++ b/dataset/src/test/scala/frameless/GroupByTests.scala @@ -16,8 +16,8 @@ class GroupByTests extends TypedDatasetSuite { widen: B => Out ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) + val A = dataset.col('a) + val B = dataset.col('b) val datasetSumByA = dataset.groupByMany(A).agg(sum(B)).collect().run.toVector.sortBy(_._1) val sumByA = data.groupBy(_.a).mapValues(_.map(_.b).map(widen).sum).toVector.sortBy(_._1) @@ -34,7 +34,7 @@ class GroupByTests extends TypedDatasetSuite { summable: CatalystSummable[A, A] ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetSum = dataset.agg(sum(A)).collect().run().toVector val listSum = data.map(_.a).sum @@ -55,8 +55,8 @@ class GroupByTests extends TypedDatasetSuite { bs: CatalystSummable[B, B] ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) + val A = dataset.col('a) + val B = dataset.col('b) val datasetSum = dataset.agg(sum(A), sum(B)).collect().run().toVector val listSumA = data.map(_.a).sum @@ -80,9 +80,9 @@ class GroupByTests extends TypedDatasetSuite { cs: CatalystSummable[C, C] ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) val datasetSum = dataset.agg(sum(A), sum(B), sum(C)).collect().run().toVector val listSumA = data.map(_.a).sum @@ -109,10 +109,10 @@ class GroupByTests extends TypedDatasetSuite { fo: CatalystOrdered[D] ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) - val D = dataset.col[D]('d) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) + val D = dataset.col('d) val datasetSum = dataset.agg(sum(A), sum(B), min(C), max(D)).collect().run().toVector val listSumA = data.map(_.a).sum @@ -139,8 +139,8 @@ class GroupByTests extends TypedDatasetSuite { widen: B => Out ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) + val A = dataset.col('a) + val B = dataset.col('b) val datasetSumByA = dataset.groupBy(A).agg(sum(B)).collect().run.toVector.sortBy(_._1) val sumByA = data.groupBy(_.a).mapValues(_.map(_.b).map(widen).sum).toVector.sortBy(_._1) @@ -157,8 +157,8 @@ class GroupByTests extends TypedDatasetSuite { B: TypedEncoder : Numeric ](data: List[X2[A, B]]): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) + val A = dataset.col('a) + val B = dataset.col('b) val datasetSumByA = dataset.groupBy(A) .mapGroups { case (a, xs) => (a, xs.map(_.b).sum) } @@ -186,9 +186,9 @@ class GroupByTests extends TypedDatasetSuite { widenc: C => OutC ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) val framelessSumBC = dataset .groupBy(A) @@ -255,9 +255,9 @@ class GroupByTests extends TypedDatasetSuite { widenc: C => OutC ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) val framelessSumC = dataset .groupBy(A,B) @@ -330,10 +330,10 @@ class GroupByTests extends TypedDatasetSuite { widend: D => OutD ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) - val D = dataset.col[D]('d) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) + val D = dataset.col('d) val datasetSumByAB = dataset .groupBy(A, B) @@ -359,9 +359,9 @@ class GroupByTests extends TypedDatasetSuite { C: TypedEncoder : Numeric ](data: List[X3[A, B, C]]): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) val datasetSumByAB = dataset .groupBy(A, B) @@ -384,7 +384,7 @@ class GroupByTests extends TypedDatasetSuite { B: TypedEncoder ](data: Vector[X2[A, B]]): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetGrouped = dataset .groupBy(A) @@ -407,7 +407,7 @@ class GroupByTests extends TypedDatasetSuite { B: TypedEncoder : Ordering ](data: Vector[X2[A, B]]): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetGrouped = dataset .groupBy(A) @@ -435,8 +435,8 @@ class GroupByTests extends TypedDatasetSuite { C: TypedEncoder : Ordering ](data: Vector[X3[A, B, C]]): Prop = { val dataset = TypedDataset.create(data) - val cA = dataset.col[A]('a) - val cB = dataset.col[B]('b) + val cA = dataset.col('a) + val cB = dataset.col('b) val datasetGrouped = dataset .groupBy(cA, cB) diff --git a/dataset/src/test/scala/frameless/SelectTests.scala b/dataset/src/test/scala/frameless/SelectTests.scala index d90466ccd..0d563bec3 100644 --- a/dataset/src/test/scala/frameless/SelectTests.scala +++ b/dataset/src/test/scala/frameless/SelectTests.scala @@ -15,7 +15,7 @@ class SelectTests extends TypedDatasetSuite { ca: ClassTag[A] ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) + val A = dataset.col('a) val dataset2 = dataset.select(A).collect().run().toVector val data2 = data.map { case X4(a, _, _, _) => a } @@ -39,8 +39,8 @@ class SelectTests extends TypedDatasetSuite { ca: ClassTag[A] ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) + val A = dataset.col('a) + val B = dataset.col('b) val dataset2 = dataset.select(A, B).collect().run().toVector val data2 = data.map { case X4(a, b, _, _) => (a, b) } @@ -64,9 +64,9 @@ class SelectTests extends TypedDatasetSuite { ca: ClassTag[A] ): Prop = { val dataset = TypedDataset.create(data) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) val dataset2 = dataset.select(A, B, C).collect().run().toVector val data2 = data.map { case X4(a, b, c, _) => (a, b, c) } @@ -90,10 +90,10 @@ class SelectTests extends TypedDatasetSuite { ca: ClassTag[A] ): Prop = { val dataset = TypedDataset.create(data) - val a1 = dataset.col[A]('a) - val a2 = dataset.col[B]('b) - val a3 = dataset.col[C]('c) - val a4 = dataset.col[D]('d) + val a1 = dataset.col('a) + val a2 = dataset.col('b) + val a3 = dataset.col('c) + val a4 = dataset.col('d) val dataset2 = dataset.select(a1, a2, a3, a4).collect().run().toVector val data2 = data.map { case X4(a, b, c, d) => (a, b, c, d) } @@ -117,10 +117,10 @@ class SelectTests extends TypedDatasetSuite { ca: ClassTag[A] ): Prop = { val dataset = TypedDataset.create(data) - val a1 = dataset.col[A]('a) - val a2 = dataset.col[B]('b) - val a3 = dataset.col[C]('c) - val a4 = dataset.col[D]('d) + val a1 = dataset.col('a) + val a2 = dataset.col('b) + val a3 = dataset.col('c) + val a4 = dataset.col('d) val dataset2 = dataset.select(a1, a2, a3, a4, a1).collect().run().toVector val data2 = data.map { case X4(a, b, c, d) => (a, b, c, d, a) } @@ -144,10 +144,10 @@ class SelectTests extends TypedDatasetSuite { ca: ClassTag[A] ): Prop = { val dataset = TypedDataset.create(data) - val a1 = dataset.col[A]('a) - val a2 = dataset.col[B]('b) - val a3 = dataset.col[C]('c) - val a4 = dataset.col[D]('d) + val a1 = dataset.col('a) + val a2 = dataset.col('b) + val a3 = dataset.col('c) + val a4 = dataset.col('d) val dataset2 = dataset.select(a1, a2, a3, a4, a1, a3).collect().run().toVector val data2 = data.map { case X4(a, b, c, d) => (a, b, c, d, a, c) } @@ -171,10 +171,10 @@ class SelectTests extends TypedDatasetSuite { ca: ClassTag[A] ): Prop = { val dataset = TypedDataset.create(data) - val a1 = dataset.col[A]('a) - val a2 = dataset.col[B]('b) - val a3 = dataset.col[C]('c) - val a4 = dataset.col[D]('d) + val a1 = dataset.col('a) + val a2 = dataset.col('b) + val a3 = dataset.col('c) + val a4 = dataset.col('d) val dataset2 = dataset.select(a1, a2, a3, a4, a1, a3, a2).collect().run().toVector val data2 = data.map { case X4(a, b, c, d) => (a, b, c, d, a, c, b) } @@ -198,10 +198,10 @@ class SelectTests extends TypedDatasetSuite { ca: ClassTag[A] ): Prop = { val dataset = TypedDataset.create(data) - val a1 = dataset.col[A]('a) - val a2 = dataset.col[B]('b) - val a3 = dataset.col[C]('c) - val a4 = dataset.col[D]('d) + val a1 = dataset.col('a) + val a2 = dataset.col('b) + val a3 = dataset.col('c) + val a4 = dataset.col('d) val dataset2 = dataset.select(a1, a2, a3, a4, a1, a3, a2, a1).collect().run().toVector val data2 = data.map { case X4(a, b, c, d) => (a, b, c, d, a, c, b, a) } @@ -225,10 +225,10 @@ class SelectTests extends TypedDatasetSuite { ca: ClassTag[A] ): Prop = { val dataset = TypedDataset.create(data) - val a1 = dataset.col[A]('a) - val a2 = dataset.col[B]('b) - val a3 = dataset.col[C]('c) - val a4 = dataset.col[D]('d) + val a1 = dataset.col('a) + val a2 = dataset.col('b) + val a3 = dataset.col('c) + val a4 = dataset.col('d) val dataset2 = dataset.select(a1, a2, a3, a4, a1, a3, a2, a1, a3).collect().run().toVector val data2 = data.map { case X4(a, b, c, d) => (a, b, c, d, a, c, b, a, c) } @@ -252,10 +252,10 @@ class SelectTests extends TypedDatasetSuite { ca: ClassTag[A] ): Prop = { val dataset = TypedDataset.create(data) - val a1 = dataset.col[A]('a) - val a2 = dataset.col[B]('b) - val a3 = dataset.col[C]('c) - val a4 = dataset.col[D]('d) + val a1 = dataset.col('a) + val a2 = dataset.col('b) + val a3 = dataset.col('c) + val a4 = dataset.col('d) val dataset2 = dataset.select(a1, a2, a3, a4, a1, a3, a2, a1, a3, a4).collect().run().toVector val data2 = data.map { case X4(a, b, c, d) => (a, b, c, d, a, c, b, a, c, d) } @@ -397,4 +397,4 @@ class SelectTests extends TypedDatasetSuite { val e = TypedDataset.create[(Int, String, Long)]((1, "a", 2L) :: (2, "b", 4L) :: (2, "b", 1L) :: Nil) illTyped("""e.select(frameless.functions.aggregate.sum(e('_1)))""") } -} \ No newline at end of file +} diff --git a/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala b/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala index 8090efb83..2cd0ca98f 100644 --- a/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala +++ b/dataset/src/test/scala/frameless/functions/AggregateFunctionsTests.scala @@ -41,7 +41,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { summer: Sum4Tests[A, Out] ): Prop = { val dataset = TypedDataset.create(xs.map(X1(_))) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetSum: List[Out] = dataset.agg(sum(A)).collect().run().toList @@ -81,7 +81,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { summer: Sum4Tests[A, Out] ): Prop = { val dataset = TypedDataset.create(xs.map(X1(_))) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetSum: List[Out] = dataset.agg(sumDistinct(A)).collect().run().toList @@ -115,7 +115,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { averager: Averager4Tests[A, Out] ): Prop = { val dataset = TypedDataset.create(xs.map(X1(_))) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetAvg: Vector[Out] = dataset.agg(avg(A)).collect().run().toVector @@ -148,7 +148,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { def prop[A: TypedEncoder : CatalystVariance : Numeric](xs: List[A]): Prop = { val numeric = implicitly[Numeric[A]] val dataset = TypedDataset.create(xs.map(X1(_))) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetStdOpt = dataset.agg(stddev(A)).collect().run().toVector.headOption val datasetVarOpt = dataset.agg(variance(A)).collect().run().toVector.headOption @@ -185,7 +185,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { test("count('a)") { def prop[A: TypedEncoder](xs: List[A]): Prop = { val dataset = TypedDataset.create(xs.map(X1(_))) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetCount = dataset.agg(count(A)).collect().run() datasetCount ?= List(xs.size.toLong) @@ -198,7 +198,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { test("max") { def prop[A: TypedEncoder: CatalystOrdered](xs: List[A])(implicit o: Ordering[A]): Prop = { val dataset = TypedDataset.create(xs.map(X1(_))) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetMax = dataset.agg(max(A)).collect().run().toList datasetMax ?= xs.reduceOption(o.max).toList @@ -215,7 +215,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { test("min") { def prop[A: TypedEncoder: CatalystOrdered](xs: List[A])(implicit o: Ordering[A]): Prop = { val dataset = TypedDataset.create(xs.map(X1(_))) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetMin = dataset.agg(min(A)).collect().run().toList @@ -233,7 +233,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { test("first") { def prop[A: TypedEncoder](xs: List[A]): Prop = { val dataset = TypedDataset.create(xs.map(X1(_))) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetFirst = dataset.agg(first(A)).collect().run().toList @@ -252,7 +252,7 @@ class AggregateFunctionsTests extends TypedDatasetSuite { test("last") { def prop[A: TypedEncoder](xs: List[A]): Prop = { val dataset = TypedDataset.create(xs.map(X1(_))) - val A = dataset.col[A]('a) + val A = dataset.col('a) val datasetLast = dataset.agg(last(A)).collect().run().toList diff --git a/dataset/src/test/scala/frameless/functions/UdfTests.scala b/dataset/src/test/scala/frameless/functions/UdfTests.scala index e63e1d67b..a92d4859c 100644 --- a/dataset/src/test/scala/frameless/functions/UdfTests.scala +++ b/dataset/src/test/scala/frameless/functions/UdfTests.scala @@ -11,7 +11,7 @@ class UdfTests extends TypedDatasetSuite { val dataset: TypedDataset[X1[A]] = TypedDataset.create(data) val u1 = udf[X1[A], A, B](f1) val u2 = dataset.makeUDF(f1) - val A = dataset.col[A]('a) + val A = dataset.col('a) // filter forces whole codegen val codegen = dataset.deserialized.filter((_:X1[A]) => true).select(u1(A)).collect().run().toVector @@ -51,9 +51,9 @@ class UdfTests extends TypedDatasetSuite { val u12 = dataset.makeUDF(f1) val u22 = dataset.makeUDF(f2) val u32 = dataset.makeUDF(f3) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) val dataset21 = dataset.select(u11(A), u21(B), u31(C)).collect().run().toVector val dataset22 = dataset.select(u12(A), u22(B), u32(C)).collect().run().toVector @@ -73,8 +73,8 @@ class UdfTests extends TypedDatasetSuite { val dataset = TypedDataset.create(data) val u1 = udf[X3[A, B, C], A, B, C](f1) val u2 = dataset.makeUDF(f1) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) + val A = dataset.col('a) + val B = dataset.col('b) val dataset21 = dataset.select(u1(A, B)).collect().run().toVector val dataset22 = dataset.select(u2(A, B)).collect().run().toVector @@ -96,9 +96,9 @@ class UdfTests extends TypedDatasetSuite { val u21 = udf[X3[A, B, C], B, C, A](f2) val u22 = dataset.makeUDF(f2) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) val dataset21 = dataset.select(u11(A, B), u21(B, C)).collect().run().toVector val dataset22 = dataset.select(u12(A, B), u22(B, C)).collect().run().toVector @@ -118,9 +118,9 @@ class UdfTests extends TypedDatasetSuite { val u1 = udf[X3[A, B, C], A, B, C, C](f) val u2 = dataset.makeUDF(f) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) val dataset21 = dataset.select(u1(A, B, C)).collect().run().toVector val dataset22 = dataset.select(u2(A, B, C)).collect().run().toVector @@ -140,9 +140,9 @@ class UdfTests extends TypedDatasetSuite { val u1 = udf[X3[A, B, C], A, B, C, A, C](f) val u2 = dataset.makeUDF(f) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) val dataset21 = dataset.select(u1(A, B, C, A)).collect().run().toVector val dataset22 = dataset.select(u2(A, B, C, A)).collect().run().toVector @@ -162,11 +162,11 @@ class UdfTests extends TypedDatasetSuite { val u1 = udf[X5[A, B, C, D, E], A, B, C, D, E, C](f) val u2 = dataset.makeUDF(f) - val A = dataset.col[A]('a) - val B = dataset.col[B]('b) - val C = dataset.col[C]('c) - val D = dataset.col[D]('d) - val E = dataset.col[E]('e) + val A = dataset.col('a) + val B = dataset.col('b) + val C = dataset.col('c) + val D = dataset.col('d) + val E = dataset.col('e) val dataset21 = dataset.select(u1(A, B, C, D, E)).collect().run().toVector val dataset22 = dataset.select(u2(A, B, C, D, E)).collect().run().toVector From e08f0e165b71aaa9d057b4510d257a343e0748e4 Mon Sep 17 00:00:00 2001 From: Olivier Blanvillain Date: Sat, 16 Sep 2017 00:51:57 +0200 Subject: [PATCH 6/6] New boolean joins using CanAccess `CanAccess[_, A with B]` indicates that in this context it is possible to access columns from both table `A` and table `B`. The first type parameter is a dummy argument used for type inference. The trick works as follows: `(df: TypedDataset[T]).col('a)` looks for a CanAccess[T, T] which is always available thanks to the `globalInstance` implicit defined above. Expression for joins (and other multi dataset operations) take an `implicit a: CanAccess[Any, U with T] =>` closure. Because the first (dummy) type parameter of `CanAccess` is contravariant, the locally defined implicit will always be preferred over `globalInstance`, which implements the desired behavior. --- .../src/main/scala/frameless/CanAccess.scala | 21 +++++++++++ .../main/scala/frameless/TypedDataset.scala | 35 ++++++++++--------- .../src/test/scala/frameless/JoinTests.scala | 14 ++++---- docs/src/main/tut/FeatureOverview.md | 2 +- 4 files changed, 48 insertions(+), 24 deletions(-) create mode 100644 dataset/src/main/scala/frameless/CanAccess.scala diff --git a/dataset/src/main/scala/frameless/CanAccess.scala b/dataset/src/main/scala/frameless/CanAccess.scala new file mode 100644 index 000000000..20ccfed8a --- /dev/null +++ b/dataset/src/main/scala/frameless/CanAccess.scala @@ -0,0 +1,21 @@ +package frameless + +/** `CanAccess[_, A with B]` indicates that in this context it is possible to + * access columns from both table `A` and table `B`. The first type parameter + * is a dummy argument used for type inference. + */ +sealed trait CanAccess[-T, X] + +object CanAccess { + private[this] val theInstance = new CanAccess[Nothing, Nothing] {} + private[frameless] def localCanAccessInstance[X]: CanAccess[Any, X] = theInstance.asInstanceOf[CanAccess[Any, X]] + + implicit def globalCanAccessInstance[X] = theInstance.asInstanceOf[CanAccess[X, X]] + // The trick works as follows: `(df: TypedDataset[T]).col('a)` looks for a + // CanAccess[T, T] which is always available thanks to the `globalInstance` + // implicit defined above. Expression for joins (and other multi dataset + // operations) take an `implicit a: CanAccess[Any, U with T] =>` closure. + // Because the first (dummy) type parameter of `CanAccess` is contravariant, + // the locally defined implicit will always be preferred over + // `globalInstance`, which implements the desired behavior. +} diff --git a/dataset/src/main/scala/frameless/TypedDataset.scala b/dataset/src/main/scala/frameless/TypedDataset.scala index 8bc149f63..ea988aeb8 100644 --- a/dataset/src/main/scala/frameless/TypedDataset.scala +++ b/dataset/src/main/scala/frameless/TypedDataset.scala @@ -2,12 +2,14 @@ package frameless import frameless.ops._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, CreateStruct, EqualTo, Literal} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Literal} +import org.apache.spark.sql.catalyst.plans.logical.Join import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter, RightOuter, FullOuter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql._ import shapeless._ import shapeless.ops.hlist.{Prepend, ToTraversable, Tupler} +import CanAccess.localCanAccessInstance /** [[TypedDataset]] is a safer interface for working with `Dataset`. * @@ -162,13 +164,14 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val * * It is statically checked that column with such name exists and has type `A`. */ - def col[A](column: Witness.Lt[Symbol])( + def col[A, X](column: Witness.Lt[Symbol])( implicit + ca: CanAccess[T, X], exists: TypedColumn.Exists[T, column.T, A], encoder: TypedEncoder[A] - ): TypedColumn[T, A] = { + ): TypedColumn[X, A] = { val colExpr = dataset.col(column.value.name).as[A](TypedExpressionEncoder[A]) - new TypedColumn[T, A](colExpr) + new TypedColumn[X, A](colExpr) } object colMany extends SingletonProductArgs { @@ -290,12 +293,12 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** 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[Boolean]) + def joinInner[U](other: TypedDataset[U])(condition: CanAccess[Any, T with U] => 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 = resolveSelfJoin(Join(leftPlan, rightPlan, Inner, Some(condition.expr))) + val join = resolveSelfJoin(Join(leftPlan, rightPlan, Inner, Some(condition(localCanAccessInstance).expr))) val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) val joinedDs = mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(T, U)]) TypedDataset.create[(T, U)](joinedDs) @@ -309,12 +312,12 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** 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[Boolean]) + def joinFull[U](other: TypedDataset[U])(condition: CanAccess[Any, T with U] => TypedColumn[T with U, Boolean]) (implicit e: TypedEncoder[(Option[T], Option[U])]): TypedDataset[(Option[T], Option[U])] = { import FramelessInternals._ val leftPlan = logicalPlan(dataset) val rightPlan = logicalPlan(other.dataset) - val join = resolveSelfJoin(Join(leftPlan, rightPlan, FullOuter, Some(condition.expr))) + val join = resolveSelfJoin(Join(leftPlan, rightPlan, FullOuter, Some(condition(localCanAccessInstance).expr))) val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) val joinedDs = mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(Option[T], Option[U])]) TypedDataset.create[(Option[T], Option[U])](joinedDs) @@ -323,12 +326,12 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** 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[Boolean]) + def joinRight[U](other: TypedDataset[U])(condition: CanAccess[Any, T with U] => TypedColumn[T with U, Boolean]) (implicit e: TypedEncoder[(Option[T], U)]): TypedDataset[(Option[T], U)] = { import FramelessInternals._ val leftPlan = logicalPlan(dataset) val rightPlan = logicalPlan(other.dataset) - val join = resolveSelfJoin(Join(leftPlan, rightPlan, RightOuter, Some(condition.expr))) + val join = resolveSelfJoin(Join(leftPlan, rightPlan, RightOuter, Some(condition(localCanAccessInstance).expr))) val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) val joinedDs = mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(Option[T], U)]) TypedDataset.create[(Option[T], U)](joinedDs) @@ -337,12 +340,12 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** 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[Boolean]) + def joinLeft[U](other: TypedDataset[U])(condition: CanAccess[Any, T with U] => TypedColumn[T with U, Boolean]) (implicit e: TypedEncoder[(T, Option[U])]): TypedDataset[(T, Option[U])] = { import FramelessInternals._ val leftPlan = logicalPlan(dataset) val rightPlan = logicalPlan(other.dataset) - val join = resolveSelfJoin(Join(leftPlan, rightPlan, LeftOuter, Some(condition.expr))) + val join = resolveSelfJoin(Join(leftPlan, rightPlan, LeftOuter, Some(condition(localCanAccessInstance).expr))) val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) val joinedDs = mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(T, Option[U])]) @@ -352,15 +355,15 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** 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[Boolean]): TypedDataset[T] = - new TypedDataset(self.dataset.join(other.dataset, condition.untyped, "leftsemi") + def joinLeftSemi[U](other: TypedDataset[U])(condition: CanAccess[Any, T with U] => TypedColumn[T with U, Boolean]): TypedDataset[T] = + new TypedDataset(self.dataset.join(other.dataset, condition(localCanAccessInstance).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[Boolean]): TypedDataset[T] = - new TypedDataset(self.dataset.join(other.dataset, condition.untyped, "leftanti") + def joinLeftAnti[U](other: TypedDataset[U])(condition: CanAccess[Any, T with U] => TypedColumn[T with U, Boolean]): TypedDataset[T] = + new TypedDataset(self.dataset.join(other.dataset, condition(localCanAccessInstance).untyped, "leftanti") .as[T](TypedExpressionEncoder(encoder))) /** Fixes SPARK-6231, for more details see original code in [[Dataset#join]] **/ diff --git a/dataset/src/test/scala/frameless/JoinTests.scala b/dataset/src/test/scala/frameless/JoinTests.scala index 5ada0a903..3b73aed64 100644 --- a/dataset/src/test/scala/frameless/JoinTests.scala +++ b/dataset/src/test/scala/frameless/JoinTests.scala @@ -13,7 +13,7 @@ class JoinTests extends TypedDatasetSuite { val leftDs = TypedDataset.create(left) val rightDs = TypedDataset.create(right) val joinedDs = leftDs - .joinRight(rightDs)(leftDs.col('a) === rightDs.col('a)) + .joinRight(rightDs)(implicit a => leftDs.col('a) === rightDs.col('a)) .collect().run().toVector.sorted val leftKeys = left.map(_.a).toSet @@ -43,7 +43,7 @@ class JoinTests extends TypedDatasetSuite { val leftDs = TypedDataset.create(left) val rightDs = TypedDataset.create(right) val joinedDs = leftDs - .joinInner(rightDs)(leftDs.col('a) === rightDs.col('a)) + .joinInner(rightDs)(implicit a => leftDs.col('a) === rightDs.col('a)) .collect().run().toVector.sorted val joined = { @@ -68,7 +68,7 @@ class JoinTests extends TypedDatasetSuite { val leftDs = TypedDataset.create(left) val rightDs = TypedDataset.create(right) val joinedDs = leftDs - .joinLeft(rightDs)(leftDs.col('a) === rightDs.col('a)) + .joinLeft(rightDs)(implicit a => leftDs.col('a) === rightDs.col('a)) .collect().run().toVector.sorted val rightKeys = right.map(_.a).toSet @@ -98,7 +98,7 @@ class JoinTests extends TypedDatasetSuite { val leftDs = TypedDataset.create(left) val rightDs = TypedDataset.create(right) val joinedDs = leftDs - .joinFull(rightDs)(leftDs.col('a) === rightDs.col('a)) + .joinFull(rightDs)(implicit a => leftDs.col('a) === rightDs.col('a)) .collect().run().toVector.sorted val rightKeys = right.map(_.a).toSet @@ -159,7 +159,7 @@ class JoinTests extends TypedDatasetSuite { val rightDs = TypedDataset.create(right) val rightKeys = right.map(_.a).toSet val joinedDs = leftDs - .joinLeftSemi(rightDs)(leftDs.col('a) === rightDs.col('a)) + .joinLeftSemi(rightDs)(implicit a => leftDs.col('a) === rightDs.col('a)) .collect().run().toVector.sorted val joined = { @@ -184,7 +184,7 @@ class JoinTests extends TypedDatasetSuite { val rightDs = TypedDataset.create(right) val rightKeys = right.map(_.a).toSet val joinedDs = leftDs - .joinLeftAnti(rightDs)(leftDs.col('a) === rightDs.col('a)) + .joinLeftAnti(rightDs)(implicit a => leftDs.col('a) === rightDs.col('a)) .collect().run().toVector.sorted val joined = { @@ -208,7 +208,7 @@ class JoinTests extends TypedDatasetSuite { val count = ds.dataset.join(ds.dataset, ds.dataset.col("a") === ds.dataset.col("a")).count() - val countDs = ds.joinInner(ds)(ds.col('a) === ds.col('a)) + val countDs = ds.joinInner(ds)(implicit a => ds.col('a) === ds.col('a)) .count().run() count ?= countDs diff --git a/docs/src/main/tut/FeatureOverview.md b/docs/src/main/tut/FeatureOverview.md index 59062c5f2..3ee023c74 100644 --- a/docs/src/main/tut/FeatureOverview.md +++ b/docs/src/main/tut/FeatureOverview.md @@ -295,7 +295,7 @@ val citiInfoTypedDS = TypedDataset.create(cityInfo) Here is how to join the population information to the apartment's dataset. ```tut:book -val withCityInfo = aptTypedDs.joinInner(citiInfoTypedDS)(aptTypedDs('city) === citiInfoTypedDS('name)) +val withCityInfo = aptTypedDs.joinInner(citiInfoTypedDS)(implicit a => aptTypedDs('city) === citiInfoTypedDS('name)) withCityInfo.show().run() ```