From d2404163b045c8cbb2427942cc94678041d87a42 Mon Sep 17 00:00:00 2001 From: Marios Iliofotou Date: Tue, 19 May 2026 22:33:44 -0700 Subject: [PATCH 1/7] Add Apache Spark 4.0 support (#787) Adds an in-tree `frameless-*-spark40` module set targeting Spark 4.0.2, cross-built for Scala 2.13 only (Spark 4 dropped 2.12) and requiring JDK 17. No external shim dependency: version-divergent Catalyst access is isolated behind FramelessInternals in a `src/main/spark-4` source overlay, mirroring the existing spark-3 / spark-3.4+ pattern. Key adaptations for Spark 4: - Column no longer wraps a Catalyst Expression; bridge through classic.ExpressionUtils.column and an eager ColumnNodeToExpressionConverter (the lazy ColumnNodeExpression is Unevaluable and hides children, which broke self-join disambiguation and codegen). - Dataset/SparkSession split into abstract API + classic impl; internal helpers downcast to classic for logicalPlan/sessionState/sqlContext. - ExpressionEncoder now takes a leading AgnosticEncoder (SPARK-49025); supply a metadata-only JavaBeanEncoder stand-in carrying the right ClassTag. - AnalysisException is errorClass-based; MapGroups gets a spark-4 variant. - joinCross re-encodes its result via TypedExpressionEncoder, consistent with the other joins. - Hide the new catalyst expressions.With from TypedColumn's wildcard import. Test harness: disable ANSI mode (Spark 4 default) so the property generators keep their wrap-around/null semantics, and strip field metadata in SchemaTests. All changes are no-ops on Spark 3.x. CI: add a JDK 17 leg and pin root-spark40 to Scala 2.13 / JDK 17. dataset-spark40 passes 414/414 tests; verified end-to-end on a 2-worker standalone Spark 4.0.2 cluster (groupBy/agg, self-join, joinWith, executor closures) to confirm cross-node serialization. Co-Authored-By: Claude Opus 4.7 (1M context) --- .github/workflows/ci.yml | 67 ++++++++-- README.md | 8 +- build.sbt | 94 ++++++++++++- .../main/scala/frameless/TypedColumn.scala | 81 ++++++------ .../frameless/TypedColumnMacroImpl.scala | 4 +- .../main/scala/frameless/TypedDataset.scala | 9 +- .../frameless/TypedDatasetForwarded.scala | 4 +- .../frameless/TypedExpressionEncoder.scala | 7 +- .../main/scala/frameless/ops/GroupByOps.scala | 8 +- .../apache/spark/sql/FramelessInternals.scala | 21 ++- .../apache/spark/sql/FramelessInternals.scala | 90 +++++++++++++ .../main/spark-4/frameless/MapGroups.scala | 21 +++ .../apache/spark/sql/FramelessInternals.scala | 124 ++++++++++++++++++ .../test/scala/frameless/SchemaTests.scala | 6 +- .../scala/frameless/TypedDatasetSuite.scala | 5 + .../frameless/forward/SQLContextTests.scala | 2 +- .../sql/rules/FramelessLitPushDownTests.scala | 5 +- 17 files changed, 480 insertions(+), 76 deletions(-) rename dataset/src/main/{scala => spark-3.4+}/org/apache/spark/sql/FramelessInternals.scala (80%) create mode 100644 dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala create mode 100644 dataset/src/main/spark-4/frameless/MapGroups.scala create mode 100644 dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index c72fb73a0..26da861c5 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -31,13 +31,23 @@ jobs: matrix: os: [ubuntu-22.04] scala: [2.13, 2.12] - java: [temurin@8] - project: [root-spark33, root-spark34, root-spark35] + java: [temurin@8, temurin@17] + project: [root-spark33, root-spark34, root-spark35, root-spark40] exclude: - scala: 2.13 project: root-spark33 - scala: 2.13 project: root-spark34 + - scala: 2.12 + project: root-spark40 + - java: temurin@17 + project: root-spark33 + - java: temurin@17 + project: root-spark34 + - java: temurin@17 + project: root-spark35 + - java: temurin@8 + project: root-spark40 runs-on: ${{ matrix.os }} timeout-minutes: 60 steps: @@ -62,6 +72,19 @@ jobs: if: matrix.java == 'temurin@8' && steps.setup-java-temurin-8.outputs.cache-hit == 'false' run: sbt +update + - name: Setup Java (temurin@17) + id: setup-java-temurin-17 + if: matrix.java == 'temurin@17' + uses: actions/setup-java@v5 + with: + distribution: temurin + java-version: 17 + cache: sbt + + - name: sbt update + if: matrix.java == 'temurin@17' && steps.setup-java-temurin-17.outputs.cache-hit == 'false' + run: sbt +update + - name: Check that workflows are up to date run: sbt githubWorkflowCheck @@ -115,6 +138,19 @@ jobs: if: matrix.java == 'temurin@8' && steps.setup-java-temurin-8.outputs.cache-hit == 'false' run: sbt +update + - name: Setup Java (temurin@17) + id: setup-java-temurin-17 + if: matrix.java == 'temurin@17' + uses: actions/setup-java@v5 + with: + distribution: temurin + java-version: 17 + cache: sbt + + - name: sbt update + if: matrix.java == 'temurin@17' && steps.setup-java-temurin-17.outputs.cache-hit == 'false' + run: sbt +update + - name: Import signing key if: env.PGP_SECRET != '' && env.PGP_PASSPHRASE == '' env: @@ -169,10 +205,23 @@ jobs: if: matrix.java == 'temurin@8' && steps.setup-java-temurin-8.outputs.cache-hit == 'false' run: sbt +update + - name: Setup Java (temurin@17) + id: setup-java-temurin-17 + if: matrix.java == 'temurin@17' + uses: actions/setup-java@v5 + with: + distribution: temurin + java-version: 17 + cache: sbt + + - name: sbt update + if: matrix.java == 'temurin@17' && steps.setup-java-temurin-17.outputs.cache-hit == 'false' + run: sbt +update + - name: Submit Dependencies uses: scalacenter/sbt-dependency-submission@v2 with: - modules-ignore: root-spark33_2.13 root-spark33_2.12 docs_2.13 docs_2.12 root-spark34_2.13 root-spark34_2.12 root-spark35_2.13 root-spark35_2.12 + modules-ignore: root-spark33_2.13 root-spark33_2.12 docs_2.13 docs_2.12 root-spark34_2.13 root-spark34_2.12 root-spark35_2.13 root-spark35_2.12 root-spark40_2.13 configs-ignore: test scala-tool scala-doc-tool test-internal site: @@ -180,7 +229,7 @@ jobs: strategy: matrix: os: [ubuntu-22.04] - java: [temurin@11] + java: [temurin@17] runs-on: ${{ matrix.os }} steps: - name: Checkout current branch (full) @@ -204,17 +253,17 @@ jobs: if: matrix.java == 'temurin@8' && steps.setup-java-temurin-8.outputs.cache-hit == 'false' run: sbt +update - - name: Setup Java (temurin@11) - id: setup-java-temurin-11 - if: matrix.java == 'temurin@11' + - name: Setup Java (temurin@17) + id: setup-java-temurin-17 + if: matrix.java == 'temurin@17' uses: actions/setup-java@v5 with: distribution: temurin - java-version: 11 + java-version: 17 cache: sbt - name: sbt update - if: matrix.java == 'temurin@11' && steps.setup-java-temurin-11.outputs.cache-hit == 'false' + if: matrix.java == 'temurin@17' && steps.setup-java-temurin-17.outputs.cache-hit == 'false' run: sbt +update - name: Generate site diff --git a/README.md b/README.md index 8e3f0a8df..bb22cf29e 100644 --- a/README.md +++ b/README.md @@ -27,6 +27,7 @@ The compatible versions of [Spark](http://spark.apache.org/) and | Frameless | Spark | Cats | Cats-Effect | Scala | |-----------|-----------------------------|----------|-------------|-------------| +| 0.17.0 | 4.0.2† / 3.5.8 / 3.4.4 / 3.3.4 | 2.x | 3.x | 2.12 / 2.13 | | 0.16.0 | 3.5.0 / 3.4.0 / 3.3.0 | 2.x | 3.x | 2.12 / 2.13 | | 0.15.0 | 3.4.0 / 3.3.0 / 3.2.2 | 2.x | 3.x | 2.12 / 2.13 | | 0.14.1 | 3.4.0 / 3.3.0 / 3.2.2 | 2.x | 3.x | 2.12 / 2.13 | @@ -46,6 +47,8 @@ The compatible versions of [Spark](http://spark.apache.org/) and _\* 0.11.0 has broken Spark 3.1.2 and 3.0.1 artifacts published._ +_† The Spark 4.0.x artifacts (`-spark40`) are published for **Scala 2.13 only** and require **JDK 17+**, since Spark 4 dropped Scala 2.12 and JDK 8/11. The default (unsuffixed) artifacts still target Spark 3.5._ + Starting 0.11 we introduced Spark cross published artifacts: * By default, frameless artifacts depend on the most recent Spark version @@ -53,9 +56,10 @@ Starting 0.11 we introduced Spark cross published artifacts: Artifact names examples: -* `frameless-dataset` (the latest Spark dependency) +* `frameless-dataset` (the default Spark 3.5.x dependency) +* `frameless-dataset-spark40` (Spark 4.0.x dependency; Scala 2.13 + JDK 17 only) +* `frameless-dataset-spark34` (Spark 3.4.x dependency) * `frameless-dataset-spark33` (Spark 3.3.x dependency) -* `frameless-dataset-spark32` (Spark 3.2.x dependency) Versions 0.5.x and 0.6.x have identical features. The first is compatible with Spark 2.2.1 and the second with 2.3.0. diff --git a/build.sbt b/build.sbt index a38f38f4d..2b628d1d4 100644 --- a/build.sbt +++ b/build.sbt @@ -1,4 +1,5 @@ val sparkVersion = "3.5.8" +val spark40Version = "4.0.2" val spark34Version = "3.4.4" val spark33Version = "3.3.4" val catsCoreVersion = "2.13.0" @@ -26,12 +27,25 @@ lazy val root = project .enablePlugins(NoPublishPlugin) .settings(crossScalaVersions := Nil) .aggregate( + `root-spark40`, `root-spark35`, `root-spark34`, `root-spark33`, docs ) +lazy val `root-spark40` = project + .in(file(".spark40")) + .enablePlugins(NoPublishPlugin) + .settings(crossScalaVersions := Seq(Scala213)) + .aggregate( + core, + `cats-spark40`, + `dataset-spark40`, + `refined-spark40`, + `ml-spark40` + ) + lazy val `root-spark35` = project .in(file(".spark35")) .enablePlugins(NoPublishPlugin) @@ -76,6 +90,15 @@ lazy val `cats-spark34` = project `dataset-spark34` % "test->test;compile->compile;provided->provided" ) +lazy val `cats-spark40` = project + .settings(name := "frameless-cats-spark40") + .settings(sourceDirectory := (cats / sourceDirectory).value) + .settings(catsSettings) + .settings(spark40Settings) + .dependsOn( + `dataset-spark40` % "test->test;compile->compile;provided->provided" + ) + lazy val `cats-spark33` = project .settings(name := "frameless-cats-spark33") .settings(sourceDirectory := (cats / sourceDirectory).value) @@ -111,6 +134,20 @@ lazy val `dataset-spark34` = project .settings(spark34Settings) .dependsOn(core % "test->test;compile->compile") +lazy val `dataset-spark40` = project + .settings(name := "frameless-dataset-spark40") + .settings(sourceDirectory := (dataset / sourceDirectory).value) + .settings( + Compile / unmanagedSourceDirectories += (dataset / baseDirectory).value / "src" / "main" / "spark-4" + ) + .settings( + Test / unmanagedSourceDirectories += (dataset / baseDirectory).value / "src" / "test" / "spark-3.3+" + ) + .settings(datasetSettings) + .settings(sparkDependencies(spark40Version)) + .settings(spark40Settings) + .dependsOn(core % "test->test;compile->compile") + lazy val `dataset-spark33` = project .settings(name := "frameless-dataset-spark33") .settings(sourceDirectory := (dataset / sourceDirectory).value) @@ -139,6 +176,15 @@ lazy val `refined-spark34` = project `dataset-spark34` % "test->test;compile->compile;provided->provided" ) +lazy val `refined-spark40` = project + .settings(name := "frameless-refined-spark40") + .settings(sourceDirectory := (refined / sourceDirectory).value) + .settings(refinedSettings) + .settings(spark40Settings) + .dependsOn( + `dataset-spark40` % "test->test;compile->compile;provided->provided" + ) + lazy val `refined-spark33` = project .settings(name := "frameless-refined-spark33") .settings(sourceDirectory := (refined / sourceDirectory).value) @@ -168,6 +214,17 @@ lazy val `ml-spark34` = project `dataset-spark34` % "test->test;compile->compile;provided->provided" ) +lazy val `ml-spark40` = project + .settings(name := "frameless-ml-spark40") + .settings(sourceDirectory := (ml / sourceDirectory).value) + .settings(mlSettings) + .settings(sparkMlDependencies(spark40Version)) + .settings(spark40Settings) + .dependsOn( + core % "test->test;compile->compile", + `dataset-spark40` % "test->test;compile->compile;provided->provided" + ) + lazy val `ml-spark33` = project .settings(name := "frameless-ml-spark33") .settings(sourceDirectory := (ml / sourceDirectory).value) @@ -352,6 +409,15 @@ lazy val framelessSettings = Seq( libraryDependencySchemes += "org.scala-lang.modules" %% "scala-xml" % VersionScheme.Always ) ++ consoleSettings +lazy val spark40Settings = Seq[Setting[_]]( + // Spark 4 dropped Scala 2.12 support; this module is 2.13-only. + crossScalaVersions := Seq(Scala213), + scalaVersion := Scala213, + tlVersionIntroduced := Map("2.13" -> "0.17.0"), + // Brand-new artifact: no previously published version to check binary compatibility against. + mimaPreviousArtifacts := Set.empty +) + lazy val spark34Settings = Seq[Setting[_]]( tlVersionIntroduced := Map("2.12" -> "0.14.1", "2.13" -> "0.14.1"), mimaPreviousArtifacts := Set( @@ -427,12 +493,32 @@ ThisBuild / developers := List( ThisBuild / tlCiReleaseBranches := Seq("master") ThisBuild / tlSitePublishBranch := Some("master") -val roots = List("root-spark33", "root-spark34", "root-spark35") +// Spark 3.x roots: 3.3/3.4 build on 2.12 only, 3.5 builds on both 2.12 and 2.13. +val spark3Roots = List("root-spark33", "root-spark34", "root-spark35") +// Spark 4.x roots: Scala 2.13 only (Spark 4 dropped 2.12). +val spark4Roots = List("root-spark40") +val roots = spark3Roots ++ spark4Roots + +// Spark 3.x builds/tests on JDK 8; Spark 4 requires JDK 17+. +val spark3Java = JavaSpec.temurin("8") +val spark4Java = JavaSpec.temurin("17") + +ThisBuild / githubWorkflowJavaVersions := Seq(spark3Java, spark4Java) ThisBuild / githubWorkflowBuildMatrixAdditions += "project" -> roots -ThisBuild / githubWorkflowBuildMatrixExclusions ++= roots.init.map { project => - MatrixExclude(Map("scala" -> "2.13", "project" -> project)) -} +ThisBuild / githubWorkflowBuildMatrixExclusions ++= + // 3.3/3.4 are 2.12-only; 3.5 builds both. Spark 4 is 2.13-only. + spark3Roots.init.map { project => + MatrixExclude(Map("scala" -> "2.13", "project" -> project)) + } ++ spark4Roots.map { project => + MatrixExclude(Map("scala" -> "2.12", "project" -> project)) + } ++ + // Pin each Spark line to its JDK: 3.x on JDK 8, 4.x on JDK 17. + spark3Roots.map { project => + MatrixExclude(Map("java" -> spark4Java.render, "project" -> project)) + } ++ spark4Roots.map { project => + MatrixExclude(Map("java" -> spark3Java.render, "project" -> project)) + } ThisBuild / githubWorkflowEnv += "SBT_OPTS" -> "-Xms1g -Xmx4g" diff --git a/dataset/src/main/scala/frameless/TypedColumn.scala b/dataset/src/main/scala/frameless/TypedColumn.scala index 0bbaf6fed..f6926c10d 100644 --- a/dataset/src/main/scala/frameless/TypedColumn.scala +++ b/dataset/src/main/scala/frameless/TypedColumn.scala @@ -7,6 +7,11 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.DecimalType import org.apache.spark.sql.{Column, FramelessInternals} +// Spark 4 added org.apache.spark.sql.catalyst.expressions.With, which the wildcard import +// above would otherwise bind in preference to frameless.With. Alias frameless.With so its +// references resolve consistently on every supported Spark version. +import frameless.{With => FWith} + import shapeless._ import shapeless.ops.record.Selector @@ -103,16 +108,16 @@ abstract class AbstractTypedColumn[T, U] def opt[X](implicit x: U <:< Option[X]): Mapper[X] = new Mapper[X] {} /** Fall back to an untyped Column */ - def untyped: Column = new Column(expr) + def untyped: Column = FramelessInternals.column(expr) - private def equalsTo[TT, W](other: ThisType[TT, U])(implicit w: With.Aux[T, TT, W]): ThisType[W, Boolean] = typed { + private def equalsTo[TT, W](other: ThisType[TT, U])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed { if (uencoder.nullable) EqualNullSafe(self.expr, other.expr) else EqualTo(self.expr, other.expr) } /** Creates a typed column of either TypedColumn or TypedAggregate from an expression. */ protected def typed[W, U1: TypedEncoder](e: Expression): ThisType[W, U1] = - typed(new Column(e)) + typed(FramelessInternals.column(e)) /** Creates a typed column of either TypedColumn or TypedAggregate. */ def typed[W, U1: TypedEncoder](c: Column): ThisType[W, U1] @@ -137,7 +142,7 @@ abstract class AbstractTypedColumn[T, U] * * apache/spark */ - def ===[TT, W](other: ThisType[TT, U])(implicit w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def ===[TT, W](other: ThisType[TT, U])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = equalsTo(other) /** Inequality test. @@ -148,7 +153,7 @@ abstract class AbstractTypedColumn[T, U] * * apache/spark */ - def =!=[TT, W](other: ThisType[TT, U])(implicit w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def =!=[TT, W](other: ThisType[TT, U])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed(Not(equalsTo(other).expr)) /** Inequality test. @@ -214,7 +219,7 @@ abstract class AbstractTypedColumn[T, U] * df(df('opt).getOrElse(df('defaultValue))) * }}} */ - def getOrElse[TT, W, Out](default: ThisType[TT, Out])(implicit i0: U =:= Option[Out], i1: With.Aux[T, TT, W]): ThisType[W, Out] = + def getOrElse[TT, W, Out](default: ThisType[TT, Out])(implicit i0: U =:= Option[Out], i1: FWith.Aux[T, TT, W]): ThisType[W, Out] = typed(Coalesce(Seq(expr, default.expr)))(default.uencoder) /** Convert an Optional column by providing a default value. @@ -235,7 +240,7 @@ abstract class AbstractTypedColumn[T, U] * * apache/spark */ - def plus[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def plus[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = typed(self.untyped.plus(other.untyped)) /** Sum of this expression and another expression. @@ -246,7 +251,7 @@ abstract class AbstractTypedColumn[T, U] * * apache/spark */ - def +[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def +[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = plus(other) /** Sum of this expression (column) with a constant. @@ -292,7 +297,7 @@ abstract class AbstractTypedColumn[T, U] * * apache/spark */ - def minus[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def minus[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = typed(self.untyped.minus(other.untyped)) /** Subtraction. Subtract the other expression from this expression. @@ -303,7 +308,7 @@ abstract class AbstractTypedColumn[T, U] * * apache/spark */ - def -[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def -[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = minus(other) /** Subtraction. Subtract the other expression from this expression. @@ -330,7 +335,7 @@ abstract class AbstractTypedColumn[T, U] (other: ThisType[TT, U]) (implicit n: CatalystNumeric[U], - w: With.Aux[T, TT, W], + w: FWith.Aux[T, TT, W], t: ClassTag[U] ): ThisType[W, U] = typed { if (t.runtimeClass == BigDecimal(0).getClass) { @@ -351,7 +356,7 @@ abstract class AbstractTypedColumn[T, U] * * apache/spark */ - def *[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: With.Aux[T, TT, W], t: ClassTag[U]): ThisType[W, U] = + def *[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W], t: ClassTag[U]): ThisType[W, U] = multiply(other) /** Multiplication of this expression a constant. @@ -369,14 +374,14 @@ abstract class AbstractTypedColumn[T, U] * * apache/spark */ - def mod[Out: TypedEncoder, TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: With.Aux[T, TT, W]): ThisType[W, Out] = + def mod[Out: TypedEncoder, TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, Out] = typed(self.untyped.mod(other.untyped)) /** Modulo (a.k.a. remainder) expression. * * apache/spark */ - def %[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def %[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = mod(other) /** Modulo (a.k.a. remainder) expression. @@ -395,7 +400,7 @@ abstract class AbstractTypedColumn[T, U] * @param other another column of the same type * apache/spark */ - def divide[Out: TypedEncoder, TT, W](other: ThisType[TT, U])(implicit n: CatalystDivisible[U, Out], w: With.Aux[T, TT, W]): ThisType[W, Out] = + def divide[Out: TypedEncoder, TT, W](other: ThisType[TT, U])(implicit n: CatalystDivisible[U, Out], w: FWith.Aux[T, TT, W]): ThisType[W, Out] = typed(self.untyped.divide(other.untyped)) /** Division this expression by another expression. @@ -407,7 +412,7 @@ abstract class AbstractTypedColumn[T, U] * @param other another column of the same type * apache/spark */ - def /[Out, TT, W](other: ThisType[TT, U])(implicit n: CatalystDivisible[U, Out], e: TypedEncoder[Out], w: With.Aux[T, TT, W]): ThisType[W, Out] = + def /[Out, TT, W](other: ThisType[TT, U])(implicit n: CatalystDivisible[U, Out], e: TypedEncoder[Out], w: FWith.Aux[T, TT, W]): ThisType[W, Out] = divide(other) /** Division this expression by another expression. @@ -455,7 +460,7 @@ abstract class AbstractTypedColumn[T, U] * @param u a constant of the same type * apache/spark */ - def bitwiseAND[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def bitwiseAND[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = typed(self.untyped.bitwiseAND(other.untyped)) /** Bitwise AND this expression and another expression (of same type). @@ -477,7 +482,7 @@ abstract class AbstractTypedColumn[T, U] * @param other a constant of the same type * apache/spark */ - def &[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def &[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = bitwiseAND(other) /** Bitwise OR this expression and another expression. @@ -499,7 +504,7 @@ abstract class AbstractTypedColumn[T, U] * @param other a constant of the same type * apache/spark */ - def bitwiseOR[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def bitwiseOR[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = typed(self.untyped.bitwiseOR(other.untyped)) /** Bitwise OR this expression and another expression (of same type). @@ -521,7 +526,7 @@ abstract class AbstractTypedColumn[T, U] * @param other a constant of the same type * apache/spark */ - def |[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def |[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = bitwiseOR(other) /** Bitwise XOR this expression and another expression. @@ -543,7 +548,7 @@ abstract class AbstractTypedColumn[T, U] * @param other a constant of the same type * apache/spark */ - def bitwiseXOR[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def bitwiseXOR[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = typed(self.untyped.bitwiseXOR(other.untyped)) /** Bitwise XOR this expression and another expression (of same type). @@ -565,7 +570,7 @@ abstract class AbstractTypedColumn[T, U] * @param other a constant of the same type * apache/spark */ - def ^[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: With.Aux[T, TT, W]): ThisType[W, U] = + def ^[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = bitwiseXOR(other) /** Casts the column to a different type. @@ -600,8 +605,8 @@ abstract class AbstractTypedColumn[T, U] def substr[TT1, TT2, W1, W2](startPos: ThisType[TT1, Int], len: ThisType[TT2, Int]) (implicit ev: U =:= String, - w1: With.Aux[T, TT1, W1], - w2: With.Aux[W1, TT2, W2]): ThisType[W2, String] = + w1: FWith.Aux[T, TT1, W1], + w2: FWith.Aux[W1, TT2, W2]): ThisType[W2, String] = typed(self.untyped.substr(startPos.untyped, len.untyped)) /** SQL like expression. Returns a boolean column based on a SQL LIKE match. @@ -651,7 +656,7 @@ abstract class AbstractTypedColumn[T, U] * @param other a column which values is used as a string that is being tested against. * apache/spark */ - def contains[TT, W](other: ThisType[TT, U])(implicit ev: U =:= String, w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def contains[TT, W](other: ThisType[TT, U])(implicit ev: U =:= String, w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed(self.untyped.contains(other.untyped)) /** String starts with another string literal. @@ -673,7 +678,7 @@ abstract class AbstractTypedColumn[T, U] * @param other a column which values is used as a prefix that is being tested against. * apache/spark */ - def startsWith[TT, W](other: ThisType[TT, U])(implicit ev: U =:= String, w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def startsWith[TT, W](other: ThisType[TT, U])(implicit ev: U =:= String, w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed(self.untyped.startsWith(other.untyped)) /** String ends with another string literal. @@ -695,7 +700,7 @@ abstract class AbstractTypedColumn[T, U] * @param other a column which values is used as a suffix that is being tested against. * apache/spark */ - def endsWith[TT, W](other: ThisType[TT, U])(implicit ev: U =:= String, w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def endsWith[TT, W](other: ThisType[TT, U])(implicit ev: U =:= String, w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed(self.untyped.endsWith(other.untyped)) /** Boolean AND. @@ -703,7 +708,7 @@ abstract class AbstractTypedColumn[T, U] * df.filter ( (df.col('a) === 1).and(df.col('b) > 5) ) * }}} */ - def and[TT, W](other: ThisType[TT, Boolean])(implicit w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def and[TT, W](other: ThisType[TT, Boolean])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed(self.untyped.and(other.untyped)) /** Boolean AND. @@ -711,7 +716,7 @@ abstract class AbstractTypedColumn[T, U] * df.filter ( df.col('a) === 1 && df.col('b) > 5) * }}} */ - def && [TT, W](other: ThisType[TT, Boolean])(implicit w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def && [TT, W](other: ThisType[TT, Boolean])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = and(other) /** Boolean OR. @@ -719,7 +724,7 @@ abstract class AbstractTypedColumn[T, U] * df.filter ( (df.col('a) === 1).or(df.col('b) > 5) ) * }}} */ - def or[TT, W](other: ThisType[TT, Boolean])(implicit w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def or[TT, W](other: ThisType[TT, Boolean])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed(self.untyped.or(other.untyped)) /** Boolean OR. @@ -727,7 +732,7 @@ abstract class AbstractTypedColumn[T, U] * df.filter ( df.col('a) === 1 || df.col('b) > 5) * }}} */ - def || [TT, W](other: ThisType[TT, Boolean])(implicit w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def || [TT, W](other: ThisType[TT, Boolean])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = or(other) /** Less than. @@ -740,7 +745,7 @@ abstract class AbstractTypedColumn[T, U] * @param other another column of the same type * apache/spark */ - def <[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def <[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed(self.untyped < other.untyped) /** Less than or equal to. @@ -753,7 +758,7 @@ abstract class AbstractTypedColumn[T, U] * @param other another column of the same type * apache/spark */ - def <=[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def <=[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed(self.untyped <= other.untyped) /** Greater than. @@ -765,7 +770,7 @@ abstract class AbstractTypedColumn[T, U] * @param other another column of the same type * apache/spark */ - def >[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def >[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed(self.untyped > other.untyped) /** Greater than or equal. @@ -777,7 +782,7 @@ abstract class AbstractTypedColumn[T, U] * @param other another column of the same type * apache/spark */ - def >=[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: With.Aux[T, TT, W]): ThisType[W, Boolean] = + def >=[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed(self.untyped >= other.untyped) /** Less than. @@ -861,8 +866,8 @@ abstract class AbstractTypedColumn[T, U] def between[TT1, TT2, W1, W2](lowerBound: ThisType[TT1, U], upperBound: ThisType[TT2, U]) (implicit i0: CatalystOrdered[U], - w0: With.Aux[T, TT1, W1], - w1: With.Aux[TT2, W1, W2] + w0: FWith.Aux[T, TT1, W1], + w1: FWith.Aux[TT2, W1, W2] ): ThisType[W2, Boolean] = typed(self.untyped.between(lowerBound.untyped, upperBound.untyped)) @@ -890,7 +895,7 @@ sealed class SortedTypedColumn[T, U](val expr: Expression)( this(FramelessInternals.expr(column)) } - def untyped: Column = new Column(expr) + def untyped: Column = FramelessInternals.column(expr) } object SortedTypedColumn { diff --git a/dataset/src/main/scala/frameless/TypedColumnMacroImpl.scala b/dataset/src/main/scala/frameless/TypedColumnMacroImpl.scala index 62fa2765d..d741dccaa 100644 --- a/dataset/src/main/scala/frameless/TypedColumnMacroImpl.scala +++ b/dataset/src/main/scala/frameless/TypedColumnMacroImpl.scala @@ -13,7 +13,7 @@ private[frameless] object TypedColumnMacroImpl { def buildExpression(path: List[String]): c.Expr[TypedColumn[T, U]] = { val columnName = path.mkString(".") - c.Expr[TypedColumn[T, U]](q"new _root_.frameless.TypedColumn[$t, $u]((org.apache.spark.sql.functions.col($columnName)).expr)") + c.Expr[TypedColumn[T, U]](q"new _root_.frameless.TypedColumn[$t, $u](_root_.org.apache.spark.sql.FramelessInternals.expr(org.apache.spark.sql.functions.col($columnName)))") } def abort(msg: String) = c.abort(c.enclosingPosition, msg) @@ -66,7 +66,7 @@ private[frameless] object TypedColumnMacroImpl { expectedRoot.forall(_ == root) && check(t, tail)) => { val colPath = tail.mkString(".") - c.Expr[TypedColumn[T, U]](q"new _root_.frameless.TypedColumn[$t, $u]((org.apache.spark.sql.functions.col($colPath)).expr)") + c.Expr[TypedColumn[T, U]](q"new _root_.frameless.TypedColumn[$t, $u](_root_.org.apache.spark.sql.FramelessInternals.expr(org.apache.spark.sql.functions.col($colPath)))") } case _ => diff --git a/dataset/src/main/scala/frameless/TypedDataset.scala b/dataset/src/main/scala/frameless/TypedDataset.scala index add2170b2..9946b457d 100644 --- a/dataset/src/main/scala/frameless/TypedDataset.scala +++ b/dataset/src/main/scala/frameless/TypedDataset.scala @@ -109,7 +109,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val val underlyingColumns = columns.toList[UntypedExpression[T]] val cols: Seq[Column] = for { (c, i) <- columns.toList[UntypedExpression[T]].zipWithIndex - } yield new Column(c.expr).as(s"_${i+1}") + } yield FramelessInternals.column(c.expr).as(s"_${i+1}") // Workaround to SPARK-20346. One alternative is to allow the result to be Vector(null) for empty DataFrames. // Another one would be to return an Option. @@ -635,7 +635,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val /** 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")) + new TypedDataset(self.dataset.joinWith(other.dataset, FramelessInternals.column(Literal(true)), "cross") + .as[(T, U)](TypedExpressionEncoder[(T, U)])) /** Computes the full outer join of `this` `Dataset` with the `other` `Dataset`, * returning a `Tuple2` for each pair where condition evaluates to true. @@ -656,7 +657,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val val rightPlan = logicalPlan(other.dataset) val join = disambiguate(Join(leftPlan, rightPlan, Inner, Some(condition.expr), JoinHint.NONE)) val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) - val joinedDs = mkDataset(dataset.sqlContext, joinedPlan, TypedExpressionEncoder[(T, U)]) + val joinedDs = mkDataset(dataset, joinedPlan, TypedExpressionEncoder[(T, U)]) TypedDataset.create[(T, U)](joinedDs) } @@ -938,7 +939,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val i3: TypedEncoder[Out] ): TypedDataset[Out] = { val base = dataset.toDF() - .select(columns.toList[UntypedExpression[T]].map(c => new Column(c.expr)):_*) + .select(columns.toList[UntypedExpression[T]].map(c => FramelessInternals.column(c.expr)):_*) val selected = base.as[Out](TypedExpressionEncoder[Out]) TypedDataset.create[Out](selected) diff --git a/dataset/src/main/scala/frameless/TypedDatasetForwarded.scala b/dataset/src/main/scala/frameless/TypedDatasetForwarded.scala index d417caf8e..119256279 100644 --- a/dataset/src/main/scala/frameless/TypedDatasetForwarded.scala +++ b/dataset/src/main/scala/frameless/TypedDatasetForwarded.scala @@ -6,7 +6,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.streaming.DataStreamWriter import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{DataFrame, DataFrameWriter, SQLContext, SparkSession} +import org.apache.spark.sql.{DataFrame, DataFrameWriter, FramelessInternals, SQLContext, SparkSession} import org.apache.spark.storage.StorageLevel import scala.util.Random @@ -34,7 +34,7 @@ trait TypedDatasetForwarded[T] { self: TypedDataset[T] => * Returns a `SQLContext` from this [[TypedDataset]]. */ def sqlContext: SQLContext = - dataset.sqlContext + FramelessInternals.sqlContext(dataset) /** * Returns the schema of this Dataset. diff --git a/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala b/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala index 5b78cd292..1c2e3bc38 100644 --- a/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala +++ b/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala @@ -1,8 +1,7 @@ package frameless -import org.apache.spark.sql.Encoder +import org.apache.spark.sql.{Encoder, FramelessInternals} import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{BoundReference, CreateNamedStruct, If} import org.apache.spark.sql.types.StructType @@ -39,10 +38,10 @@ object TypedExpressionEncoder { } } - new ExpressionEncoder[T]( + FramelessInternals.expressionEncoder[T]( objSerializer = serializer, objDeserializer = encoder.fromCatalyst(out), - clsTag = encoder.classTag + classTag = encoder.classTag ) } } diff --git a/dataset/src/main/scala/frameless/ops/GroupByOps.scala b/dataset/src/main/scala/frameless/ops/GroupByOps.scala index 3feeaca59..ccbacce8c 100644 --- a/dataset/src/main/scala/frameless/ops/GroupByOps.scala +++ b/dataset/src/main/scala/frameless/ops/GroupByOps.scala @@ -144,7 +144,7 @@ private[ops] abstract class AggregatingOps[T, TK <: HList, K <: HList, KT] i7: TypedEncoder[Out1], i8: ToTraversable.Aux[TC, List, UntypedExpression[T]] ): TypedDataset[Out1] = { - def expr(c: UntypedExpression[T]): Column = new Column(c.expr) + def expr(c: UntypedExpression[T]): Column = FramelessInternals.column(c.expr) val groupByExprs = groupedBy.toList[UntypedExpression[T]].map(expr) val aggregates = @@ -191,7 +191,7 @@ private[ops] abstract class AggregatingOps[T, TK <: HList, K <: HList, KT] )(TypedExpressionEncoder[KT], TypedExpressionEncoder[T], TypedExpressionEncoder[U]) val groupedAndFlatMapped = FramelessInternals.mkDataset( - self.dataset.sqlContext, + self.dataset, mapGroups, TypedExpressionEncoder[U] ) @@ -201,7 +201,7 @@ private[ops] abstract class AggregatingOps[T, TK <: HList, K <: HList, KT] } private def retainGroupColumns: Boolean = { - self.dataset.sqlContext.getConf("spark.sql.retainGroupColumns", "true").toBoolean + FramelessInternals.getConf(self.dataset, "spark.sql.retainGroupColumns", "true").toBoolean } def pivot[P: CatalystPivotable](pivotColumn: TypedColumn[T, P]): PivotNotValues[T, TK, P] = @@ -243,7 +243,7 @@ final case class Pivot[T, GroupedColumns <: HList, PivotType, Values <: HList]( case x :: xs => f(x) :: mapAny(xs)(f) } - val aggCols: Seq[Column] = mapAny(aggrColumns)(x => new Column(x.asInstanceOf[TypedAggregate[_,_]].expr)) + val aggCols: Seq[Column] = mapAny(aggrColumns)(x => FramelessInternals.column(x.asInstanceOf[TypedAggregate[_,_]].expr)) val tmp = ds.dataset.toDF() .groupBy(mapAny(groupedBy)(_.asInstanceOf[TypedColumn[_, _]].untyped): _*) .pivot(pivotedBy.untyped.toString, mapAny(values)(identity)) diff --git a/dataset/src/main/scala/org/apache/spark/sql/FramelessInternals.scala b/dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala similarity index 80% rename from dataset/src/main/scala/org/apache/spark/sql/FramelessInternals.scala rename to dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala index 5459230d4..81c221968 100644 --- a/dataset/src/main/scala/org/apache/spark/sql/FramelessInternals.scala +++ b/dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala @@ -4,6 +4,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.{Alias, CreateStruct} import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} @@ -22,6 +23,9 @@ object FramelessInternals { } } + /** Wraps a Catalyst [[Expression]] into a [[Column]]. */ + def column(e: Expression): Column = new Column(e) + def expr(column: Column): Expression = column.expr def logicalPlan(ds: Dataset[_]): LogicalPlan = ds.logicalPlan @@ -29,6 +33,11 @@ object FramelessInternals { def executePlan(ds: Dataset[_], plan: LogicalPlan): QueryExecution = ds.sparkSession.sessionState.executePlan(plan) + def sqlContext(ds: Dataset[_]): SQLContext = ds.sqlContext + + def getConf(ds: Dataset[_], key: String, default: String): String = + ds.sqlContext.getConf(key, default) + 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) @@ -40,12 +49,20 @@ object FramelessInternals { ), joined.analyzed) } - def mkDataset[T](sqlContext: SQLContext, plan: LogicalPlan, encoder: Encoder[T]): Dataset[T] = - new Dataset(sqlContext, plan, encoder) + def mkDataset[T](source: Dataset[_], plan: LogicalPlan, encoder: Encoder[T]): Dataset[T] = + new Dataset(source.sparkSession, plan, encoder) def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = Dataset.ofRows(sparkSession, logicalPlan) + /** Builds an [[ExpressionEncoder]] from frameless' own serializer/deserializer expressions. */ + def expressionEncoder[T]( + objSerializer: Expression, + objDeserializer: Expression, + classTag: ClassTag[T] + ): ExpressionEncoder[T] = + new ExpressionEncoder[T](objSerializer, objDeserializer, classTag) + // 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/main/spark-3/org/apache/spark/sql/FramelessInternals.scala b/dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala new file mode 100644 index 000000000..81c221968 --- /dev/null +++ b/dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala @@ -0,0 +1,90 @@ +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.{Alias, CreateStruct} +import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.ObjectType +import scala.reflect.ClassTag + +object FramelessInternals { + def objectTypeFor[A](implicit classTag: ClassTag[A]): ObjectType = ObjectType(classTag.runtimeClass) + + def resolveExpr(ds: Dataset[_], colNames: Seq[String]): NamedExpression = { + ds.toDF().queryExecution.analyzed.resolve(colNames, ds.sparkSession.sessionState.analyzer.resolver).getOrElse { + throw new AnalysisException( + s"""Cannot resolve column name "$colNames" among (${ds.schema.fieldNames.mkString(", ")})""") + } + } + + /** Wraps a Catalyst [[Expression]] into a [[Column]]. */ + def column(e: Expression): Column = new Column(e) + + def expr(column: Column): Expression = column.expr + + def logicalPlan(ds: Dataset[_]): LogicalPlan = ds.logicalPlan + + def executePlan(ds: Dataset[_], plan: LogicalPlan): QueryExecution = + ds.sparkSession.sessionState.executePlan(plan) + + def sqlContext(ds: Dataset[_]): SQLContext = ds.sqlContext + + def getConf(ds: Dataset[_], key: String, default: String): String = + ds.sqlContext.getConf(key, default) + + 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](source: Dataset[_], plan: LogicalPlan, encoder: Encoder[T]): Dataset[T] = + new Dataset(source.sparkSession, plan, encoder) + + def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = + Dataset.ofRows(sparkSession, logicalPlan) + + /** Builds an [[ExpressionEncoder]] from frameless' own serializer/deserializer expressions. */ + def expressionEncoder[T]( + objSerializer: Expression, + objDeserializer: Expression, + classTag: ClassTag[T] + ): ExpressionEncoder[T] = + new ExpressionEncoder[T](objSerializer, objDeserializer, classTag) + + // because org.apache.spark.sql.types.UserDefinedType is private[spark] + type UserDefinedType[A >: Null] = org.apache.spark.sql.types.UserDefinedType[A] + + // below only tested in SelfJoinTests.colLeft and colRight are equivalent to col outside of joins + // - via files (codegen) forces doGenCode eval. + /** Expression to tag columns from the left hand side of join expression. */ + case class DisambiguateLeft[T](tagged: Expression) extends Expression with NonSQLExpression { + def eval(input: InternalRow): Any = tagged.eval(input) + def nullable: Boolean = false + def children: Seq[Expression] = tagged :: Nil + def dataType: DataType = tagged.dataType + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = tagged.genCode(ctx) + protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(newChildren.head) + } + + /** Expression to tag columns from the right hand side of join expression. */ + case class DisambiguateRight[T](tagged: Expression) extends Expression with NonSQLExpression { + def eval(input: InternalRow): Any = tagged.eval(input) + def nullable: Boolean = false + def children: Seq[Expression] = tagged :: Nil + def dataType: DataType = tagged.dataType + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = tagged.genCode(ctx) + protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(newChildren.head) + } +} diff --git a/dataset/src/main/spark-4/frameless/MapGroups.scala b/dataset/src/main/spark-4/frameless/MapGroups.scala new file mode 100644 index 000000000..6856acba4 --- /dev/null +++ b/dataset/src/main/spark-4/frameless/MapGroups.scala @@ -0,0 +1,21 @@ +package frameless + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, MapGroups => SMapGroups} + +object MapGroups { + def apply[K: Encoder, T: Encoder, U: Encoder]( + func: (K, Iterator[T]) => TraversableOnce[U], + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + child: LogicalPlan + ): LogicalPlan = + SMapGroups( + func, + groupingAttributes, + dataAttributes, + Seq(), // #698 - no order given + child + ) +} diff --git a/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala b/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala new file mode 100644 index 000000000..478c27457 --- /dev/null +++ b/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala @@ -0,0 +1,124 @@ +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.{Alias, CreateStruct} +import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.JavaBeanEncoder +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.classic.{Dataset => ClassicDataset, SparkSession => ClassicSparkSession, ExpressionUtils, ColumnNodeToExpressionConverter} +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.ObjectType +import scala.reflect.ClassTag + +/** + * Spark 4 split `Dataset`/`SparkSession`/`Column` into abstract API types + * (`org.apache.spark.sql.*`) and concrete implementations (`org.apache.spark.sql.classic.*`). + * The `Dataset`/`SparkSession` instances frameless holds are always the `classic` + * implementations at runtime, so the internal-only helpers below downcast to reach the + * `logicalPlan`/`sessionState`/`sqlContext` members that the abstract API no longer exposes. + * `Column` no longer wraps a Catalyst `Expression`; `classic.ExpressionUtils` is Spark's + * own bridge between the two. + */ +object FramelessInternals { + def objectTypeFor[A](implicit classTag: ClassTag[A]): ObjectType = ObjectType(classTag.runtimeClass) + + private def classic(ds: Dataset[_]): ClassicDataset[_] = + ds.asInstanceOf[ClassicDataset[_]] + + def resolveExpr(ds: Dataset[_], colNames: Seq[String]): NamedExpression = { + val cds = classic(ds) + cds.queryExecution.analyzed + .resolve(colNames, cds.sparkSession.sessionState.analyzer.resolver) + .getOrElse { + throw new AnalysisException( + errorClass = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + messageParameters = Map("objectName" -> colNames.mkString(".")) + ) + } + } + + /** Wraps a Catalyst [[Expression]] into a [[Column]] (Spark 4 bridge). */ + def column(e: Expression): Column = ExpressionUtils.column(e) + + /** + * Extracts the Catalyst [[Expression]] from a [[Column]]. + * + * `ExpressionUtils.expression` would return a lazy `ColumnNodeExpression` wrapper, which is + * `Unevaluable` and exposes no children. frameless builds join plans by hand and rewrites + * disambiguation markers via `Expression.transform`, both of which require a real, traversable + * expression tree - so convert the column's node eagerly instead. + */ + def expr(column: Column): Expression = ColumnNodeToExpressionConverter(column.node) + + def logicalPlan(ds: Dataset[_]): LogicalPlan = classic(ds).logicalPlan + + def executePlan(ds: Dataset[_], plan: LogicalPlan): QueryExecution = + classic(ds).sparkSession.sessionState.executePlan(plan) + + def sqlContext(ds: Dataset[_]): SQLContext = classic(ds).sqlContext + + def getConf(ds: Dataset[_], key: String, default: String): String = + classic(ds).sparkSession.conf.get(key, default) + + 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](source: Dataset[_], plan: LogicalPlan, encoder: Encoder[T]): Dataset[T] = + new ClassicDataset[T](classic(source).sparkSession, plan, encoder) + + def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = + ClassicDataset.ofRows(sparkSession.asInstanceOf[ClassicSparkSession], logicalPlan) + + /** + * Builds an [[ExpressionEncoder]] from frameless' own serializer/deserializer expressions. + * + * Spark 4's `ExpressionEncoder` takes a leading `AgnosticEncoder` (SPARK-49025), but it is + * only read for its `clsTag` and an Option-wrapping check - the serializer, deserializer and + * schema are all derived from the expressions frameless supplies. A minimal `JavaBeanEncoder` + * carrying the right `ClassTag` is therefore a correct, metadata-only stand-in. + */ + def expressionEncoder[T]( + objSerializer: Expression, + objDeserializer: Expression, + classTag: ClassTag[T] + ): ExpressionEncoder[T] = + new ExpressionEncoder[T](JavaBeanEncoder(classTag, Nil), objSerializer, objDeserializer) + + // because org.apache.spark.sql.types.UserDefinedType is private[spark] + type UserDefinedType[A >: Null] = org.apache.spark.sql.types.UserDefinedType[A] + + // below only tested in SelfJoinTests.colLeft and colRight are equivalent to col outside of joins + // - via files (codegen) forces doGenCode eval. + /** Expression to tag columns from the left hand side of join expression. */ + case class DisambiguateLeft[T](tagged: Expression) extends Expression with NonSQLExpression { + def eval(input: InternalRow): Any = tagged.eval(input) + def nullable: Boolean = false + def children: Seq[Expression] = tagged :: Nil + def dataType: DataType = tagged.dataType + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = tagged.genCode(ctx) + protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(newChildren.head) + } + + /** Expression to tag columns from the right hand side of join expression. */ + case class DisambiguateRight[T](tagged: Expression) extends Expression with NonSQLExpression { + def eval(input: InternalRow): Any = tagged.eval(input) + def nullable: Boolean = false + def children: Seq[Expression] = tagged :: Nil + def dataType: DataType = tagged.dataType + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = tagged.genCode(ctx) + protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(newChildren.head) + } +} diff --git a/dataset/src/test/scala/frameless/SchemaTests.scala b/dataset/src/test/scala/frameless/SchemaTests.scala index 92fd33057..7dac7215a 100644 --- a/dataset/src/test/scala/frameless/SchemaTests.scala +++ b/dataset/src/test/scala/frameless/SchemaTests.scala @@ -2,7 +2,7 @@ package frameless import frameless.functions.aggregate._ import frameless.functions._ -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{Metadata, StructType} import org.scalacheck.Prop import org.scalacheck.Prop._ import org.scalatest.matchers.should.Matchers @@ -10,7 +10,9 @@ import org.scalatest.matchers.should.Matchers class SchemaTests extends TypedDatasetSuite with Matchers { def structToNonNullable(struct: StructType): StructType = { - StructType(struct.fields.map( f => f.copy(nullable = false))) + // Spark 4 attaches metadata to aggregate output columns; clear it (and nullability) + // so comparisons consider only field names and types. No-op on Spark 3.x. + StructType(struct.fields.map( f => f.copy(nullable = false, metadata = Metadata.empty))) } def prop[A](dataset: TypedDataset[A], ignoreNullable: Boolean = false): Prop = { diff --git a/dataset/src/test/scala/frameless/TypedDatasetSuite.scala b/dataset/src/test/scala/frameless/TypedDatasetSuite.scala index 8a4697835..1314fe74e 100644 --- a/dataset/src/test/scala/frameless/TypedDatasetSuite.scala +++ b/dataset/src/test/scala/frameless/TypedDatasetSuite.scala @@ -33,6 +33,11 @@ trait SparkTesting { self: BeforeAndAfterAll => .setAppName("test") .set("spark.ui.enabled", "false") .set("spark.app.id", appID) + // Spark 4 enables ANSI SQL mode by default, which makes the property-based + // generators (extreme numeric values, malformed date strings) raise overflow / + // cast errors instead of the wrap-around / null semantics these tests assert. + // No-op on Spark 3.x, where ANSI is already disabled by default. + .set("spark.sql.ansi.enabled", "false") private var s: SparkSession = _ diff --git a/dataset/src/test/scala/frameless/forward/SQLContextTests.scala b/dataset/src/test/scala/frameless/forward/SQLContextTests.scala index 700f29b05..454dca8a2 100644 --- a/dataset/src/test/scala/frameless/forward/SQLContextTests.scala +++ b/dataset/src/test/scala/frameless/forward/SQLContextTests.scala @@ -8,7 +8,7 @@ class SQLContextTests extends TypedDatasetSuite { def prop[A: TypedEncoder](data: Vector[A]): Prop = { val dataset = TypedDataset.create[A](data) - dataset.sqlContext =? dataset.dataset.sqlContext + dataset.sqlContext =? org.apache.spark.sql.FramelessInternals.sqlContext(dataset.dataset) } check(forAll(prop[Int] _)) diff --git a/dataset/src/test/spark-3.3+/frameless/sql/rules/FramelessLitPushDownTests.scala b/dataset/src/test/spark-3.3+/frameless/sql/rules/FramelessLitPushDownTests.scala index 36a443fb5..5b09f1cc3 100644 --- a/dataset/src/test/spark-3.3+/frameless/sql/rules/FramelessLitPushDownTests.scala +++ b/dataset/src/test/spark-3.3+/frameless/sql/rules/FramelessLitPushDownTests.scala @@ -2,14 +2,15 @@ package frameless.sql.rules import frameless._ import frameless.functions.Lit -import org.apache.spark.sql.catalyst.util.DateTimeUtils.{currentTimestamp, microsToInstant} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToInstant import org.apache.spark.sql.sources.{EqualTo, GreaterThanOrEqual, IsNotNull} import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import java.time.Instant class FramelessLitPushDownTests extends SQLRulesSuite { - private val now: Long = currentTimestamp() + // micros since epoch; DateTimeUtils.currentTimestamp was removed in Spark 4. + private val now: Long = System.currentTimeMillis() * 1000L test("java.sql.Timestamp push-down") { val expected = java.sql.Timestamp.from(microsToInstant(now)) From d68fcc35d34e5c2342e75669fbf060ff01f8735f Mon Sep 17 00:00:00 2001 From: Marios Iliofotou Date: Tue, 19 May 2026 22:38:44 -0700 Subject: [PATCH 2/7] Apply scalafmt formatting Co-Authored-By: Claude Opus 4.7 (1M context) --- build.sbt | 12 +- .../main/scala/frameless/TypedColumn.scala | 1777 ++++++++----- .../frameless/TypedColumnMacroImpl.scala | 60 +- .../main/scala/frameless/TypedDataset.scala | 2244 ++++++++++------- .../frameless/TypedDatasetForwarded.scala | 499 ++-- .../frameless/TypedExpressionEncoder.scala | 27 +- .../main/scala/frameless/ops/GroupByOps.scala | 334 ++- .../apache/spark/sql/FramelessInternals.scala | 87 +- .../apache/spark/sql/FramelessInternals.scala | 87 +- .../main/spark-4/frameless/MapGroups.scala | 16 +- .../apache/spark/sql/FramelessInternals.scala | 94 +- .../test/scala/frameless/SchemaTests.scala | 17 +- .../scala/frameless/TypedDatasetSuite.scala | 51 +- .../frameless/forward/SQLContextTests.scala | 5 +- .../sql/rules/FramelessLitPushDownTests.scala | 13 +- 15 files changed, 3222 insertions(+), 2101 deletions(-) diff --git a/build.sbt b/build.sbt index 2b628d1d4..a47bc51c6 100644 --- a/build.sbt +++ b/build.sbt @@ -514,11 +514,11 @@ ThisBuild / githubWorkflowBuildMatrixExclusions ++= } ++ spark4Roots.map { project => MatrixExclude(Map("scala" -> "2.12", "project" -> project)) } ++ - // Pin each Spark line to its JDK: 3.x on JDK 8, 4.x on JDK 17. - spark3Roots.map { project => - MatrixExclude(Map("java" -> spark4Java.render, "project" -> project)) - } ++ spark4Roots.map { project => - MatrixExclude(Map("java" -> spark3Java.render, "project" -> project)) - } + // Pin each Spark line to its JDK: 3.x on JDK 8, 4.x on JDK 17. + spark3Roots.map { project => + MatrixExclude(Map("java" -> spark4Java.render, "project" -> project)) + } ++ spark4Roots.map { project => + MatrixExclude(Map("java" -> spark3Java.render, "project" -> project)) + } ThisBuild / githubWorkflowEnv += "SBT_OPTS" -> "-Xms1g -Xmx4g" diff --git a/dataset/src/main/scala/frameless/TypedColumn.scala b/dataset/src/main/scala/frameless/TypedColumn.scala index f6926c10d..2888d8608 100644 --- a/dataset/src/main/scala/frameless/TypedColumn.scala +++ b/dataset/src/main/scala/frameless/TypedColumn.scala @@ -1,16 +1,16 @@ package frameless -import frameless.functions.{litAggr, lit => flit} +import frameless.functions.{ litAggr, lit => flit } import frameless.syntax._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.DecimalType -import org.apache.spark.sql.{Column, FramelessInternals} +import org.apache.spark.sql.{ Column, FramelessInternals } // Spark 4 added org.apache.spark.sql.catalyst.expressions.With, which the wildcard import // above would otherwise bind in preference to frameless.With. Alias frameless.With so its // references resolve consistently on every supported Spark version. -import frameless.{With => FWith} +import frameless.{ With => FWith } import shapeless._ import shapeless.ops.record.Selector @@ -26,91 +26,121 @@ sealed trait UntypedExpression[T] { override def toString: String = expr.toString() } -/** Expression used in `select`-like constructions. - */ -sealed class TypedColumn[T, U](expr: Expression)( - implicit val uenc: TypedEncoder[U] -) extends AbstractTypedColumn[T, U](expr) { +/** + * Expression used in `select`-like constructions. + */ +sealed class TypedColumn[T, U]( + expr: Expression + )(implicit + val uenc: TypedEncoder[U]) + extends AbstractTypedColumn[T, U](expr) { type ThisType[A, B] = TypedColumn[A, B] - def this(column: Column)(implicit uencoder: TypedEncoder[U]) = + def this( + column: Column + )(implicit + uencoder: TypedEncoder[U] + ) = this(FramelessInternals.expr(column)) - override def typed[W, U1: TypedEncoder](c: Column): TypedColumn[W, U1] = c.typedColumn + override def typed[W, U1: TypedEncoder](c: Column): TypedColumn[W, U1] = + c.typedColumn override def lit[U1: TypedEncoder](c: U1): TypedColumn[T, U1] = flit(c) } -/** Expression used in `agg`-like constructions. - */ -sealed class TypedAggregate[T, U](expr: Expression)( - implicit val uenc: TypedEncoder[U] -) extends AbstractTypedColumn[T, U](expr) { +/** + * Expression used in `agg`-like constructions. + */ +sealed class TypedAggregate[T, U]( + expr: Expression + )(implicit + val uenc: TypedEncoder[U]) + extends AbstractTypedColumn[T, U](expr) { type ThisType[A, B] = TypedAggregate[A, B] - def this(column: Column)(implicit uencoder: TypedEncoder[U]) = { + def this( + column: Column + )(implicit + uencoder: TypedEncoder[U] + ) = { this(FramelessInternals.expr(column)) } - override def typed[W, U1: TypedEncoder](c: Column): TypedAggregate[W, U1] = c.typedAggregate + override def typed[W, U1: TypedEncoder](c: Column): TypedAggregate[W, U1] = + c.typedAggregate override def lit[U1: TypedEncoder](c: U1): TypedAggregate[T, U1] = litAggr(c) } -/** Generic representation of a typed column. A typed column can either be a [[TypedAggregate]] or - * a [[frameless.TypedColumn]]. - * - * Documentation marked "apache/spark" is thanks to apache/spark Contributors - * at https://github.com/apache/spark, licensed under Apache v2.0 available at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * @tparam T phantom type representing the dataset on which this columns is - * selected. When `T = A with B` the selection is on either A or B. - * @tparam U type of column - */ -abstract class AbstractTypedColumn[T, U] - (val expr: Expression) - (implicit val uencoder: TypedEncoder[U]) +/** + * Generic representation of a typed column. A typed column can either be a [[TypedAggregate]] or + * a [[frameless.TypedColumn]]. + * + * Documentation marked "apache/spark" is thanks to apache/spark Contributors + * at https://github.com/apache/spark, licensed under Apache v2.0 available at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * @tparam T phantom type representing the dataset on which this columns is + * selected. When `T = A with B` the selection is on either A or B. + * @tparam U type of column + */ +abstract class AbstractTypedColumn[T, U]( + val expr: Expression + )(implicit + val uencoder: TypedEncoder[U]) extends UntypedExpression[T] { self => type ThisType[A, B] <: AbstractTypedColumn[A, B] - /** A helper class to make to simplify working with Optional fields. - * - * {{{ - * val x: TypedColumn[Option[Int]] = _ - * x.opt.map(_*2) // This only compiles if the type of x is Option[X] (in this example X is of type Int) - * }}} - * - * @note Known issue: map() will NOT work when the applied function is a udf(). - * It will compile and then throw a runtime error. - **/ + /** + * A helper class to make to simplify working with Optional fields. + * + * {{{ + * val x: TypedColumn[Option[Int]] = _ + * x.opt.map(_*2) // This only compiles if the type of x is Option[X] (in this example X is of type Int) + * }}} + * + * @note Known issue: map() will NOT work when the applied function is a udf(). + * It will compile and then throw a runtime error. + */ trait Mapper[X] { - def map[G, OutputType[_,_]](u: ThisType[T, X] => OutputType[T,G]) - (implicit - ev: OutputType[T,G] <:< AbstractTypedColumn[T, G] + + def map[G, OutputType[_, _]]( + u: ThisType[T, X] => OutputType[T, G] + )(implicit + ev: OutputType[T, G] <:< AbstractTypedColumn[T, G] ): OutputType[T, Option[G]] = { - u(self.asInstanceOf[ThisType[T, X]]).asInstanceOf[OutputType[T, Option[G]]] + u(self.asInstanceOf[ThisType[T, X]]) + .asInstanceOf[OutputType[T, Option[G]]] } } - /** Makes it easier to work with Optional columns. It returns an instance of `Mapper[X]` - * where `X` is type of the unwrapped Optional. E.g., in the case of `Option[Long]`, - * `X` is of type Long. - * - * {{{ - * val x: TypedColumn[Option[Int]] = _ - * x.opt.map(_*2) - * }}} - * */ - def opt[X](implicit x: U <:< Option[X]): Mapper[X] = new Mapper[X] {} + /** + * Makes it easier to work with Optional columns. It returns an instance of `Mapper[X]` + * where `X` is type of the unwrapped Optional. E.g., in the case of `Option[Long]`, + * `X` is of type Long. + * + * {{{ + * val x: TypedColumn[Option[Int]] = _ + * x.opt.map(_*2) + * }}} + */ + def opt[X]( + implicit + x: U <:< Option[X] + ): Mapper[X] = new Mapper[X] {} /** Fall back to an untyped Column */ def untyped: Column = FramelessInternals.column(expr) - private def equalsTo[TT, W](other: ThisType[TT, U])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = typed { + private def equalsTo[TT, W]( + other: ThisType[TT, U] + )(implicit + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed { if (uencoder.nullable) EqualNullSafe(self.expr, other.expr) else EqualTo(self.expr, other.expr) } @@ -125,773 +155,1125 @@ abstract class AbstractTypedColumn[T, U] /** Creates a typed column of either TypedColumn or TypedAggregate. */ def lit[U1: TypedEncoder](c: U1): ThisType[T, U1] - /** Equality test. - * {{{ - * df.filter( df.col('a) === 1 ) - * }}} - * - * apache/spark - */ + /** + * Equality test. + * {{{ + * df.filter( df.col('a) === 1 ) + * }}} + * + * apache/spark + */ def ===(u: U): ThisType[T, Boolean] = equalsTo(lit(u)) - /** Equality test. - * {{{ - * df.filter( df.col('a) === df.col('b) ) - * }}} - * - * apache/spark - */ - def ===[TT, W](other: ThisType[TT, U])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * Equality test. + * {{{ + * df.filter( df.col('a) === df.col('b) ) + * }}} + * + * apache/spark + */ + def ===[TT, W]( + other: ThisType[TT, U] + )(implicit + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = equalsTo(other) - /** Inequality test. - * - * {{{ - * df.filter(df.col('a) =!= df.col('b)) - * }}} - * - * apache/spark - */ - def =!=[TT, W](other: ThisType[TT, U])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * Inequality test. + * + * {{{ + * df.filter(df.col('a) =!= df.col('b)) + * }}} + * + * apache/spark + */ + def =!=[TT, W]( + other: ThisType[TT, U] + )(implicit + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed(Not(equalsTo(other).expr)) - /** Inequality test. - * - * {{{ - * df.filter(df.col('a) =!= "a") - * }}} - * - * apache/spark - */ + /** + * Inequality test. + * + * {{{ + * df.filter(df.col('a) =!= "a") + * }}} + * + * apache/spark + */ def =!=(u: U): ThisType[T, Boolean] = typed(Not(equalsTo(lit(u)).expr)) - /** True if the current expression is an Option and it's None. - * - * apache/spark - */ - def isNone(implicit i0: U <:< Option[_]): ThisType[T, Boolean] = + /** + * True if the current expression is an Option and it's None. + * + * apache/spark + */ + def isNone( + implicit + i0: U <:< Option[_] + ): ThisType[T, Boolean] = typed(IsNull(expr)) - /** True if the current expression is an Option and it's not None. - * - * apache/spark - */ - def isNotNone(implicit i0: U <:< Option[_]): ThisType[T, Boolean] = + /** + * True if the current expression is an Option and it's not None. + * + * apache/spark + */ + def isNotNone( + implicit + i0: U <:< Option[_] + ): ThisType[T, Boolean] = typed(IsNotNull(expr)) - /** True if the current expression is a fractional number and is not NaN. - * - * apache/spark - */ - def isNaN(implicit n: CatalystNaN[U]): ThisType[T, Boolean] = + /** + * True if the current expression is a fractional number and is not NaN. + * + * apache/spark + */ + def isNaN( + implicit + n: CatalystNaN[U] + ): ThisType[T, Boolean] = typed(self.untyped.isNaN) /** - * True if the value for this optional column `exists` as expected - * (see `Option.exists`). - * - * {{{ - * df.col('opt).isSome(_ === someOtherCol) - * }}} - */ - def isSome[V](exists: ThisType[T, V] => ThisType[T, Boolean])(implicit i0: U <:< Option[V]): ThisType[T, Boolean] = someOr[V](exists, false) + * True if the value for this optional column `exists` as expected + * (see `Option.exists`). + * + * {{{ + * df.col('opt).isSome(_ === someOtherCol) + * }}} + */ + def isSome[V]( + exists: ThisType[T, V] => ThisType[T, Boolean] + )(implicit + i0: U <:< Option[V] + ): ThisType[T, Boolean] = someOr[V](exists, false) /** - * True if the value for this optional column `exists` as expected, - * or is `None`. (see `Option.forall`). - * - * {{{ - * df.col('opt).isSomeOrNone(_ === someOtherCol) - * }}} - */ - def isSomeOrNone[V](exists: ThisType[T, V] => ThisType[T, Boolean])(implicit i0: U <:< Option[V]): ThisType[T, Boolean] = someOr[V](exists, true) - - private def someOr[V](exists: ThisType[T, V] => ThisType[T, Boolean], default: Boolean)(implicit i0: U <:< Option[V]): ThisType[T, Boolean] = { + * True if the value for this optional column `exists` as expected, + * or is `None`. (see `Option.forall`). + * + * {{{ + * df.col('opt).isSomeOrNone(_ === someOtherCol) + * }}} + */ + def isSomeOrNone[V]( + exists: ThisType[T, V] => ThisType[T, Boolean] + )(implicit + i0: U <:< Option[V] + ): ThisType[T, Boolean] = someOr[V](exists, true) + + private def someOr[V]( + exists: ThisType[T, V] => ThisType[T, Boolean], + default: Boolean + )(implicit + i0: U <:< Option[V] + ): ThisType[T, Boolean] = { val defaultExpr = if (default) Literal.TrueLiteral else Literal.FalseLiteral typed(Coalesce(Seq(opt(i0).map(exists).expr, defaultExpr))) } - /** Convert an Optional column by providing a default value. - * - * {{{ - * df(df('opt).getOrElse(df('defaultValue))) - * }}} - */ - def getOrElse[TT, W, Out](default: ThisType[TT, Out])(implicit i0: U =:= Option[Out], i1: FWith.Aux[T, TT, W]): ThisType[W, Out] = + /** + * Convert an Optional column by providing a default value. + * + * {{{ + * df(df('opt).getOrElse(df('defaultValue))) + * }}} + */ + def getOrElse[TT, W, Out]( + default: ThisType[TT, Out] + )(implicit + i0: U =:= Option[Out], + i1: FWith.Aux[T, TT, W] + ): ThisType[W, Out] = typed(Coalesce(Seq(expr, default.expr)))(default.uencoder) - /** Convert an Optional column by providing a default value. - * - * {{{ - * df( df('opt).getOrElse(defaultConstant) ) - * }}} - */ - def getOrElse[Out: TypedEncoder](default: Out)(implicit i0: U =:= Option[Out]): ThisType[T, Out] = + /** + * Convert an Optional column by providing a default value. + * + * {{{ + * df( df('opt).getOrElse(defaultConstant) ) + * }}} + */ + def getOrElse[Out: TypedEncoder]( + default: Out + )(implicit + i0: U =:= Option[Out] + ): ThisType[T, Out] = getOrElse(lit[Out](default)) - /** Sum of this expression and another expression. - * - * {{{ - * // The following selects the sum of a person's height and weight. - * people.select( people.col('height) plus people.col('weight) ) - * }}} - * - * apache/spark - */ - def plus[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Sum of this expression and another expression. + * + * {{{ + * // The following selects the sum of a person's height and weight. + * people.select( people.col('height) plus people.col('weight) ) + * }}} + * + * apache/spark + */ + def plus[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystNumeric[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = typed(self.untyped.plus(other.untyped)) - /** Sum of this expression and another expression. - * {{{ - * // The following selects the sum of a person's height and weight. - * people.select( people.col('height) + people.col('weight) ) - * }}} - * - * apache/spark - */ - def +[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Sum of this expression and another expression. + * {{{ + * // The following selects the sum of a person's height and weight. + * people.select( people.col('height) + people.col('weight) ) + * }}} + * + * apache/spark + */ + def +[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystNumeric[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = plus(other) - /** Sum of this expression (column) with a constant. - * {{{ - * // The following selects the sum of a person's height and weight. - * people.select( people('height) + 2 ) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def +(u: U)(implicit n: CatalystNumeric[U]): ThisType[T, U] = + /** + * Sum of this expression (column) with a constant. + * {{{ + * // The following selects the sum of a person's height and weight. + * people.select( people('height) + 2 ) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def +( + u: U + )(implicit + n: CatalystNumeric[U] + ): ThisType[T, U] = typed(self.untyped.plus(u)) /** - * Inversion of boolean expression, i.e. NOT. - * {{{ - * // Select rows that are not active (isActive === false) - * df.filter( !df('isActive) ) - * }}} - * - * apache/spark - */ - def unary_!(implicit i0: U <:< Boolean): ThisType[T, Boolean] = + * Inversion of boolean expression, i.e. NOT. + * {{{ + * // Select rows that are not active (isActive === false) + * df.filter( !df('isActive) ) + * }}} + * + * apache/spark + */ + def unary_!( + implicit + i0: U <:< Boolean + ): ThisType[T, Boolean] = typed(!untyped) - /** Unary minus, i.e. negate the expression. - * {{{ - * // Select the amount column and negates all values. - * df.select( -df('amount) ) - * }}} - * - * apache/spark - */ - def unary_-(implicit n: CatalystNumeric[U]): ThisType[T, U] = + /** + * Unary minus, i.e. negate the expression. + * {{{ + * // Select the amount column and negates all values. + * df.select( -df('amount) ) + * }}} + * + * apache/spark + */ + def unary_-( + implicit + n: CatalystNumeric[U] + ): ThisType[T, U] = typed(-self.untyped) - /** Subtraction. Subtract the other expression from this expression. - * {{{ - * // The following selects the difference between people's height and their weight. - * people.select( people.col('height) minus people.col('weight) ) - * }}} - * - * apache/spark - */ - def minus[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Subtraction. Subtract the other expression from this expression. + * {{{ + * // The following selects the difference between people's height and their weight. + * people.select( people.col('height) minus people.col('weight) ) + * }}} + * + * apache/spark + */ + def minus[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystNumeric[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = typed(self.untyped.minus(other.untyped)) - /** Subtraction. Subtract the other expression from this expression. - * {{{ - * // The following selects the difference between people's height and their weight. - * people.select( people.col('height) - people.col('weight) ) - * }}} - * - * apache/spark - */ - def -[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Subtraction. Subtract the other expression from this expression. + * {{{ + * // The following selects the difference between people's height and their weight. + * people.select( people.col('height) - people.col('weight) ) + * }}} + * + * apache/spark + */ + def -[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystNumeric[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = minus(other) - /** Subtraction. Subtract the other expression from this expression. - * {{{ - * // The following selects the difference between people's height and their weight. - * people.select( people('height) - 1 ) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def -(u: U)(implicit n: CatalystNumeric[U]): ThisType[T, U] = + /** + * Subtraction. Subtract the other expression from this expression. + * {{{ + * // The following selects the difference between people's height and their weight. + * people.select( people('height) - 1 ) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def -( + u: U + )(implicit + n: CatalystNumeric[U] + ): ThisType[T, U] = typed(self.untyped.minus(u)) - /** Multiplication of this expression and another expression. - * {{{ - * // The following multiplies a person's height by their weight. - * people.select( people.col('height) multiply people.col('weight) ) - * }}} - * - * apache/spark - */ - def multiply[TT, W] - (other: ThisType[TT, U]) - (implicit + /** + * Multiplication of this expression and another expression. + * {{{ + * // The following multiplies a person's height by their weight. + * people.select( people.col('height) multiply people.col('weight) ) + * }}} + * + * apache/spark + */ + def multiply[TT, W]( + other: ThisType[TT, U] + )(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W], t: ClassTag[U] ): ThisType[W, U] = typed { - if (t.runtimeClass == BigDecimal(0).getClass) { - // That's apparently the only way to get sound multiplication. - // See https://issues.apache.org/jira/browse/SPARK-22036 - val dt = DecimalType(20, 14) - self.untyped.cast(dt).multiply(other.untyped.cast(dt)) - } else { - self.untyped.multiply(other.untyped) - } + if (t.runtimeClass == BigDecimal(0).getClass) { + // That's apparently the only way to get sound multiplication. + // See https://issues.apache.org/jira/browse/SPARK-22036 + val dt = DecimalType(20, 14) + self.untyped.cast(dt).multiply(other.untyped.cast(dt)) + } else { + self.untyped.multiply(other.untyped) } + } - /** Multiplication of this expression and another expression. - * {{{ - * // The following multiplies a person's height by their weight. - * people.select( people.col('height) * people.col('weight) ) - * }}} - * - * apache/spark - */ - def *[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W], t: ClassTag[U]): ThisType[W, U] = + /** + * Multiplication of this expression and another expression. + * {{{ + * // The following multiplies a person's height by their weight. + * people.select( people.col('height) * people.col('weight) ) + * }}} + * + * apache/spark + */ + def *[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystNumeric[U], + w: FWith.Aux[T, TT, W], + t: ClassTag[U] + ): ThisType[W, U] = multiply(other) - /** Multiplication of this expression a constant. - * {{{ - * // The following multiplies a person's height by their weight. - * people.select( people.col('height) * people.col('weight) ) - * }}} - * - * apache/spark - */ - def *(u: U)(implicit n: CatalystNumeric[U]): ThisType[T, U] = + /** + * Multiplication of this expression a constant. + * {{{ + * // The following multiplies a person's height by their weight. + * people.select( people.col('height) * people.col('weight) ) + * }}} + * + * apache/spark + */ + def *( + u: U + )(implicit + n: CatalystNumeric[U] + ): ThisType[T, U] = typed(self.untyped.multiply(u)) - /** Modulo (a.k.a. remainder) expression. - * - * apache/spark - */ - def mod[Out: TypedEncoder, TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, Out] = + /** + * Modulo (a.k.a. remainder) expression. + * + * apache/spark + */ + def mod[Out: TypedEncoder, TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystNumeric[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, Out] = typed(self.untyped.mod(other.untyped)) - /** Modulo (a.k.a. remainder) expression. - * - * apache/spark - */ - def %[TT, W](other: ThisType[TT, U])(implicit n: CatalystNumeric[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Modulo (a.k.a. remainder) expression. + * + * apache/spark + */ + def %[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystNumeric[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = mod(other) - /** Modulo (a.k.a. remainder) expression. - * - * apache/spark - */ - def %(u: U)(implicit n: CatalystNumeric[U]): ThisType[T, U] = + /** + * Modulo (a.k.a. remainder) expression. + * + * apache/spark + */ + def %( + u: U + )(implicit + n: CatalystNumeric[U] + ): ThisType[T, U] = typed(self.untyped.mod(u)) - /** Division this expression by another expression. - * {{{ - * // The following divides a person's height by their weight. - * people.select( people('height) / people('weight) ) - * }}} - * - * @param other another column of the same type - * apache/spark - */ - def divide[Out: TypedEncoder, TT, W](other: ThisType[TT, U])(implicit n: CatalystDivisible[U, Out], w: FWith.Aux[T, TT, W]): ThisType[W, Out] = + /** + * Division this expression by another expression. + * {{{ + * // The following divides a person's height by their weight. + * people.select( people('height) / people('weight) ) + * }}} + * + * @param other another column of the same type + * apache/spark + */ + def divide[Out: TypedEncoder, TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystDivisible[U, Out], + w: FWith.Aux[T, TT, W] + ): ThisType[W, Out] = typed(self.untyped.divide(other.untyped)) - /** Division this expression by another expression. - * {{{ - * // The following divides a person's height by their weight. - * people.select( people('height) / people('weight) ) - * }}} - * - * @param other another column of the same type - * apache/spark - */ - def /[Out, TT, W](other: ThisType[TT, U])(implicit n: CatalystDivisible[U, Out], e: TypedEncoder[Out], w: FWith.Aux[T, TT, W]): ThisType[W, Out] = + /** + * Division this expression by another expression. + * {{{ + * // The following divides a person's height by their weight. + * people.select( people('height) / people('weight) ) + * }}} + * + * @param other another column of the same type + * apache/spark + */ + def /[Out, TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystDivisible[U, Out], + e: TypedEncoder[Out], + w: FWith.Aux[T, TT, W] + ): ThisType[W, Out] = divide(other) - /** Division this expression by another expression. - * {{{ - * // The following divides a person's height by their weight. - * people.select( people('height) / 2 ) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def /(u: U)(implicit n: CatalystNumeric[U]): ThisType[T, Double] = + /** + * Division this expression by another expression. + * {{{ + * // The following divides a person's height by their weight. + * people.select( people('height) / 2 ) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def /( + u: U + )(implicit + n: CatalystNumeric[U] + ): ThisType[T, Double] = typed(self.untyped.divide(u)) - /** Returns a descending ordering used in sorting - * - * apache/spark - */ - def desc(implicit catalystOrdered: CatalystOrdered[U]): SortedTypedColumn[T, U] = + /** + * Returns a descending ordering used in sorting + * + * apache/spark + */ + def desc( + implicit + catalystOrdered: CatalystOrdered[U] + ): SortedTypedColumn[T, U] = new SortedTypedColumn[T, U](untyped.desc) - /** Returns an ascending ordering used in sorting - * - * apache/spark - */ - def asc(implicit catalystOrdered: CatalystOrdered[U]): SortedTypedColumn[T, U] = + /** + * Returns an ascending ordering used in sorting + * + * apache/spark + */ + def asc( + implicit + catalystOrdered: CatalystOrdered[U] + ): SortedTypedColumn[T, U] = new SortedTypedColumn[T, U](untyped.asc) - /** Bitwise AND this expression and another expression. - * {{{ - * df.select(df.col('colA) bitwiseAND (df.col('colB))) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def bitwiseAND(u: U)(implicit n: CatalystBitwise[U]): ThisType[T, U] = + /** + * Bitwise AND this expression and another expression. + * {{{ + * df.select(df.col('colA) bitwiseAND (df.col('colB))) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def bitwiseAND( + u: U + )(implicit + n: CatalystBitwise[U] + ): ThisType[T, U] = typed(self.untyped.bitwiseAND(u)) - /** Bitwise AND this expression and another expression. - * {{{ - * df.select(df.col('colA) bitwiseAND (df.col('colB))) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def bitwiseAND[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Bitwise AND this expression and another expression. + * {{{ + * df.select(df.col('colA) bitwiseAND (df.col('colB))) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def bitwiseAND[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystBitwise[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = typed(self.untyped.bitwiseAND(other.untyped)) - /** Bitwise AND this expression and another expression (of same type). - * {{{ - * df.select(df.col('colA).cast[Int] & -1) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def &(u: U)(implicit n: CatalystBitwise[U]): ThisType[T, U] = + /** + * Bitwise AND this expression and another expression (of same type). + * {{{ + * df.select(df.col('colA).cast[Int] & -1) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def &( + u: U + )(implicit + n: CatalystBitwise[U] + ): ThisType[T, U] = bitwiseAND(u) - /** Bitwise AND this expression and another expression. - * {{{ - * df.select(df.col('colA) & (df.col('colB))) - * }}} - * - * @param other a constant of the same type - * apache/spark - */ - def &[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Bitwise AND this expression and another expression. + * {{{ + * df.select(df.col('colA) & (df.col('colB))) + * }}} + * + * @param other a constant of the same type + * apache/spark + */ + def &[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystBitwise[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = bitwiseAND(other) - /** Bitwise OR this expression and another expression. - * {{{ - * df.select(df.col('colA) bitwiseOR (df.col('colB))) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def bitwiseOR(u: U)(implicit n: CatalystBitwise[U]): ThisType[T, U] = + /** + * Bitwise OR this expression and another expression. + * {{{ + * df.select(df.col('colA) bitwiseOR (df.col('colB))) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def bitwiseOR( + u: U + )(implicit + n: CatalystBitwise[U] + ): ThisType[T, U] = typed(self.untyped.bitwiseOR(u)) - /** Bitwise OR this expression and another expression. - * {{{ - * df.select(df.col('colA) bitwiseOR (df.col('colB))) - * }}} - * - * @param other a constant of the same type - * apache/spark - */ - def bitwiseOR[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Bitwise OR this expression and another expression. + * {{{ + * df.select(df.col('colA) bitwiseOR (df.col('colB))) + * }}} + * + * @param other a constant of the same type + * apache/spark + */ + def bitwiseOR[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystBitwise[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = typed(self.untyped.bitwiseOR(other.untyped)) - /** Bitwise OR this expression and another expression (of same type). - * {{{ - * df.select(df.col('colA).cast[Long] | 1L) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def |(u: U)(implicit n: CatalystBitwise[U]): ThisType[T, U] = + /** + * Bitwise OR this expression and another expression (of same type). + * {{{ + * df.select(df.col('colA).cast[Long] | 1L) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def |( + u: U + )(implicit + n: CatalystBitwise[U] + ): ThisType[T, U] = bitwiseOR(u) - /** Bitwise OR this expression and another expression. - * {{{ - * df.select(df.col('colA) | (df.col('colB))) - * }}} - * - * @param other a constant of the same type - * apache/spark - */ - def |[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Bitwise OR this expression and another expression. + * {{{ + * df.select(df.col('colA) | (df.col('colB))) + * }}} + * + * @param other a constant of the same type + * apache/spark + */ + def |[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystBitwise[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = bitwiseOR(other) - /** Bitwise XOR this expression and another expression. - * {{{ - * df.select(df.col('colA) bitwiseXOR (df.col('colB))) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def bitwiseXOR(u: U)(implicit n: CatalystBitwise[U]): ThisType[T, U] = + /** + * Bitwise XOR this expression and another expression. + * {{{ + * df.select(df.col('colA) bitwiseXOR (df.col('colB))) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def bitwiseXOR( + u: U + )(implicit + n: CatalystBitwise[U] + ): ThisType[T, U] = typed(self.untyped.bitwiseXOR(u)) - /** Bitwise XOR this expression and another expression. - * {{{ - * df.select(df.col('colA) bitwiseXOR (df.col('colB))) - * }}} - * - * @param other a constant of the same type - * apache/spark - */ - def bitwiseXOR[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Bitwise XOR this expression and another expression. + * {{{ + * df.select(df.col('colA) bitwiseXOR (df.col('colB))) + * }}} + * + * @param other a constant of the same type + * apache/spark + */ + def bitwiseXOR[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystBitwise[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = typed(self.untyped.bitwiseXOR(other.untyped)) - /** Bitwise XOR this expression and another expression (of same type). - * {{{ - * df.select(df.col('colA).cast[Long] ^ 1L) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def ^(u: U)(implicit n: CatalystBitwise[U]): ThisType[T, U] = + /** + * Bitwise XOR this expression and another expression (of same type). + * {{{ + * df.select(df.col('colA).cast[Long] ^ 1L) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def ^( + u: U + )(implicit + n: CatalystBitwise[U] + ): ThisType[T, U] = bitwiseXOR(u) - /** Bitwise XOR this expression and another expression. - * {{{ - * df.select(df.col('colA) ^ (df.col('colB))) - * }}} - * - * @param other a constant of the same type - * apache/spark - */ - def ^[TT, W](other: ThisType[TT, U])(implicit n: CatalystBitwise[U], w: FWith.Aux[T, TT, W]): ThisType[W, U] = + /** + * Bitwise XOR this expression and another expression. + * {{{ + * df.select(df.col('colA) ^ (df.col('colB))) + * }}} + * + * @param other a constant of the same type + * apache/spark + */ + def ^[TT, W]( + other: ThisType[TT, U] + )(implicit + n: CatalystBitwise[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, U] = bitwiseXOR(other) - /** Casts the column to a different type. - * {{{ - * df.select(df('a).cast[Int]) - * }}} - */ - def cast[A: TypedEncoder](implicit c: CatalystCast[U, A]): ThisType[T, A] = + /** + * Casts the column to a different type. + * {{{ + * df.select(df('a).cast[Int]) + * }}} + */ + def cast[A: TypedEncoder]( + implicit + c: CatalystCast[U, A] + ): ThisType[T, A] = typed(self.untyped.cast(TypedEncoder[A].catalystRepr)) /** - * An expression that returns a substring - * {{{ - * df.select(df('a).substr(0, 5)) - * }}} - * - * @param startPos starting position - * @param len length of the substring - */ - def substr(startPos: Int, len: Int)(implicit ev: U =:= String): ThisType[T, String] = + * An expression that returns a substring + * {{{ + * df.select(df('a).substr(0, 5)) + * }}} + * + * @param startPos starting position + * @param len length of the substring + */ + def substr( + startPos: Int, + len: Int + )(implicit + ev: U =:= String + ): ThisType[T, String] = typed(self.untyped.substr(startPos, len)) /** - * An expression that returns a substring - * {{{ - * df.select(df('a).substr(df('b), df('c))) - * }}} - * - * @param startPos expression for the starting position - * @param len expression for the length of the substring - */ - def substr[TT1, TT2, W1, W2](startPos: ThisType[TT1, Int], len: ThisType[TT2, Int]) - (implicit - ev: U =:= String, - w1: FWith.Aux[T, TT1, W1], - w2: FWith.Aux[W1, TT2, W2]): ThisType[W2, String] = + * An expression that returns a substring + * {{{ + * df.select(df('a).substr(df('b), df('c))) + * }}} + * + * @param startPos expression for the starting position + * @param len expression for the length of the substring + */ + def substr[TT1, TT2, W1, W2]( + startPos: ThisType[TT1, Int], + len: ThisType[TT2, Int] + )(implicit + ev: U =:= String, + w1: FWith.Aux[T, TT1, W1], + w2: FWith.Aux[W1, TT2, W2] + ): ThisType[W2, String] = typed(self.untyped.substr(startPos.untyped, len.untyped)) - /** SQL like expression. Returns a boolean column based on a SQL LIKE match. - * {{{ - * val ds = TypedDataset.create(X2("foo", "bar") :: Nil) - * // true - * ds.select(ds('a).like("foo")) - * - * // Selected column has value "bar" - * ds.select(when(ds('a).like("f"), ds('a)).otherwise(ds('b)) - * }}} - * apache/spark - */ - def like(literal: String)(implicit ev: U =:= String): ThisType[T, Boolean] = + /** + * SQL like expression. Returns a boolean column based on a SQL LIKE match. + * {{{ + * val ds = TypedDataset.create(X2("foo", "bar") :: Nil) + * // true + * ds.select(ds('a).like("foo")) + * + * // Selected column has value "bar" + * ds.select(when(ds('a).like("f"), ds('a)).otherwise(ds('b)) + * }}} + * apache/spark + */ + def like( + literal: String + )(implicit + ev: U =:= String + ): ThisType[T, Boolean] = typed(self.untyped.like(literal)) - /** SQL RLIKE expression (LIKE with Regex). Returns a boolean column based on a regex match. - * {{{ - * val ds = TypedDataset.create(X1("foo") :: Nil) - * // true - * ds.select(ds('a).rlike("foo")) - * - * // true - * ds.select(ds('a).rlike(".*)) - * }}} - * apache/spark - */ - def rlike(literal: String)(implicit ev: U =:= String): ThisType[T, Boolean] = + /** + * SQL RLIKE expression (LIKE with Regex). Returns a boolean column based on a regex match. + * {{{ + * val ds = TypedDataset.create(X1("foo") :: Nil) + * // true + * ds.select(ds('a).rlike("foo")) + * + * // true + * ds.select(ds('a).rlike(".*)) + * }}} + * apache/spark + */ + def rlike( + literal: String + )(implicit + ev: U =:= String + ): ThisType[T, Boolean] = typed(self.untyped.rlike(literal)) - /** String contains another string literal. - * {{{ - * df.filter ( df.col('a).contains("foo") ) - * }}} - * - * @param other a string that is being tested against. - * apache/spark - */ - def contains(other: String)(implicit ev: U =:= String): ThisType[T, Boolean] = + /** + * String contains another string literal. + * {{{ + * df.filter ( df.col('a).contains("foo") ) + * }}} + * + * @param other a string that is being tested against. + * apache/spark + */ + def contains( + other: String + )(implicit + ev: U =:= String + ): ThisType[T, Boolean] = typed(self.untyped.contains(other)) - /** String contains. - * {{{ - * df.filter ( df.col('a).contains(df.col('b) ) - * }}} - * - * @param other a column which values is used as a string that is being tested against. - * apache/spark - */ - def contains[TT, W](other: ThisType[TT, U])(implicit ev: U =:= String, w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * String contains. + * {{{ + * df.filter ( df.col('a).contains(df.col('b) ) + * }}} + * + * @param other a column which values is used as a string that is being tested against. + * apache/spark + */ + def contains[TT, W]( + other: ThisType[TT, U] + )(implicit + ev: U =:= String, + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed(self.untyped.contains(other.untyped)) - /** String starts with another string literal. - * {{{ - * df.filter ( df.col('a).startsWith("foo") - * }}} - * - * @param other a prefix that is being tested against. - * apache/spark - */ - def startsWith(other: String)(implicit ev: U =:= String): ThisType[T, Boolean] = + /** + * String starts with another string literal. + * {{{ + * df.filter ( df.col('a).startsWith("foo") + * }}} + * + * @param other a prefix that is being tested against. + * apache/spark + */ + def startsWith( + other: String + )(implicit + ev: U =:= String + ): ThisType[T, Boolean] = typed(self.untyped.startsWith(other)) - /** String starts with. - * {{{ - * df.filter ( df.col('a).startsWith(df.col('b)) - * }}} - * - * @param other a column which values is used as a prefix that is being tested against. - * apache/spark - */ - def startsWith[TT, W](other: ThisType[TT, U])(implicit ev: U =:= String, w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * String starts with. + * {{{ + * df.filter ( df.col('a).startsWith(df.col('b)) + * }}} + * + * @param other a column which values is used as a prefix that is being tested against. + * apache/spark + */ + def startsWith[TT, W]( + other: ThisType[TT, U] + )(implicit + ev: U =:= String, + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed(self.untyped.startsWith(other.untyped)) - /** String ends with another string literal. - * {{{ - * df.filter ( df.col('a).endsWith("foo") - * }}} - * - * @param other a suffix that is being tested against. - * apache/spark - */ - def endsWith(other: String)(implicit ev: U =:= String): ThisType[T, Boolean] = + /** + * String ends with another string literal. + * {{{ + * df.filter ( df.col('a).endsWith("foo") + * }}} + * + * @param other a suffix that is being tested against. + * apache/spark + */ + def endsWith( + other: String + )(implicit + ev: U =:= String + ): ThisType[T, Boolean] = typed(self.untyped.endsWith(other)) - /** String ends with. - * {{{ - * df.filter ( df.col('a).endsWith(df.col('b)) - * }}} - * - * @param other a column which values is used as a suffix that is being tested against. - * apache/spark - */ - def endsWith[TT, W](other: ThisType[TT, U])(implicit ev: U =:= String, w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * String ends with. + * {{{ + * df.filter ( df.col('a).endsWith(df.col('b)) + * }}} + * + * @param other a column which values is used as a suffix that is being tested against. + * apache/spark + */ + def endsWith[TT, W]( + other: ThisType[TT, U] + )(implicit + ev: U =:= String, + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed(self.untyped.endsWith(other.untyped)) - /** Boolean AND. - * {{{ - * df.filter ( (df.col('a) === 1).and(df.col('b) > 5) ) - * }}} - */ - def and[TT, W](other: ThisType[TT, Boolean])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * Boolean AND. + * {{{ + * df.filter ( (df.col('a) === 1).and(df.col('b) > 5) ) + * }}} + */ + def and[TT, W]( + other: ThisType[TT, Boolean] + )(implicit + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed(self.untyped.and(other.untyped)) - /** Boolean AND. - * {{{ - * df.filter ( df.col('a) === 1 && df.col('b) > 5) - * }}} - */ - def && [TT, W](other: ThisType[TT, Boolean])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * Boolean AND. + * {{{ + * df.filter ( df.col('a) === 1 && df.col('b) > 5) + * }}} + */ + def &&[TT, W]( + other: ThisType[TT, Boolean] + )(implicit + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = and(other) - /** Boolean OR. - * {{{ - * df.filter ( (df.col('a) === 1).or(df.col('b) > 5) ) - * }}} - */ - def or[TT, W](other: ThisType[TT, Boolean])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * Boolean OR. + * {{{ + * df.filter ( (df.col('a) === 1).or(df.col('b) > 5) ) + * }}} + */ + def or[TT, W]( + other: ThisType[TT, Boolean] + )(implicit + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed(self.untyped.or(other.untyped)) - /** Boolean OR. - * {{{ - * df.filter ( df.col('a) === 1 || df.col('b) > 5) - * }}} - */ - def || [TT, W](other: ThisType[TT, Boolean])(implicit w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * Boolean OR. + * {{{ + * df.filter ( df.col('a) === 1 || df.col('b) > 5) + * }}} + */ + def ||[TT, W]( + other: ThisType[TT, Boolean] + )(implicit + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = or(other) - /** Less than. - * - * {{{ - * // The following selects people younger than the maxAge column. - * df.select(df('age) < df('maxAge) ) - * }}} - * - * @param other another column of the same type - * apache/spark - */ - def <[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * Less than. + * + * {{{ + * // The following selects people younger than the maxAge column. + * df.select(df('age) < df('maxAge) ) + * }}} + * + * @param other another column of the same type + * apache/spark + */ + def <[TT, W]( + other: ThisType[TT, U] + )(implicit + i0: CatalystOrdered[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed(self.untyped < other.untyped) - /** Less than or equal to. - * - * {{{ - * // The following selects people younger or equal than the maxAge column. - * df.select(df('age) <= df('maxAge) - * }}} - * - * @param other another column of the same type - * apache/spark - */ - def <=[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * Less than or equal to. + * + * {{{ + * // The following selects people younger or equal than the maxAge column. + * df.select(df('age) <= df('maxAge) + * }}} + * + * @param other another column of the same type + * apache/spark + */ + def <=[TT, W]( + other: ThisType[TT, U] + )(implicit + i0: CatalystOrdered[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed(self.untyped <= other.untyped) - /** Greater than. - * {{{ - * // The following selects people older than the maxAge column. - * df.select( df('age) > df('maxAge) ) - * }}} - * - * @param other another column of the same type - * apache/spark - */ - def >[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * Greater than. + * {{{ + * // The following selects people older than the maxAge column. + * df.select( df('age) > df('maxAge) ) + * }}} + * + * @param other another column of the same type + * apache/spark + */ + def >[TT, W]( + other: ThisType[TT, U] + )(implicit + i0: CatalystOrdered[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed(self.untyped > other.untyped) - /** Greater than or equal. - * {{{ - * // The following selects people older or equal than the maxAge column. - * df.select( df('age) >= df('maxAge) ) - * }}} - * - * @param other another column of the same type - * apache/spark - */ - def >=[TT, W](other: ThisType[TT, U])(implicit i0: CatalystOrdered[U], w: FWith.Aux[T, TT, W]): ThisType[W, Boolean] = + /** + * Greater than or equal. + * {{{ + * // The following selects people older or equal than the maxAge column. + * df.select( df('age) >= df('maxAge) ) + * }}} + * + * @param other another column of the same type + * apache/spark + */ + def >=[TT, W]( + other: ThisType[TT, U] + )(implicit + i0: CatalystOrdered[U], + w: FWith.Aux[T, TT, W] + ): ThisType[W, Boolean] = typed(self.untyped >= other.untyped) - /** Less than. - * {{{ - * // The following selects people younger than 21. - * df.select( df('age) < 21 ) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def <(u: U)(implicit i0: CatalystOrdered[U]): ThisType[T, Boolean] = + /** + * Less than. + * {{{ + * // The following selects people younger than 21. + * df.select( df('age) < 21 ) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def <( + u: U + )(implicit + i0: CatalystOrdered[U] + ): ThisType[T, Boolean] = typed(self.untyped < lit(u)(self.uencoder).untyped) - /** Less than or equal to. - * {{{ - * // The following selects people younger than 22. - * df.select( df('age) <= 2 ) - * }}} - * - * @param u a constant of the same type - * apache/spark - */ - def <=(u: U)(implicit i0: CatalystOrdered[U]): ThisType[T, Boolean] = + /** + * Less than or equal to. + * {{{ + * // The following selects people younger than 22. + * df.select( df('age) <= 2 ) + * }}} + * + * @param u a constant of the same type + * apache/spark + */ + def <=( + u: U + )(implicit + i0: CatalystOrdered[U] + ): ThisType[T, Boolean] = typed(self.untyped <= lit(u)(self.uencoder).untyped) - /** Greater than. - * {{{ - * // The following selects people older than 21. - * df.select( df('age) > 21 ) - * }}} - * - * @param u another column of the same type - * apache/spark - */ - def >(u: U)(implicit i0: CatalystOrdered[U]): ThisType[T, Boolean] = + /** + * Greater than. + * {{{ + * // The following selects people older than 21. + * df.select( df('age) > 21 ) + * }}} + * + * @param u another column of the same type + * apache/spark + */ + def >( + u: U + )(implicit + i0: CatalystOrdered[U] + ): ThisType[T, Boolean] = typed(self.untyped > lit(u)(self.uencoder).untyped) - /** Greater than or equal. - * {{{ - * // The following selects people older than 20. - * df.select( df('age) >= 21 ) - * }}} - * - * @param u another column of the same type - * apache/spark - */ - def >=(u: U)(implicit i0: CatalystOrdered[U]): ThisType[T, Boolean] = + /** + * Greater than or equal. + * {{{ + * // The following selects people older than 20. + * df.select( df('age) >= 21 ) + * }}} + * + * @param u another column of the same type + * apache/spark + */ + def >=( + u: U + )(implicit + i0: CatalystOrdered[U] + ): ThisType[T, Boolean] = typed(self.untyped >= lit(u)(self.uencoder).untyped) /** - * Returns true if the value of this column is contained in of the arguments. - * {{{ - * // The following selects people with age 15, 20, or 30. - * df.select( df('age).isin(15, 20, 30) ) - * }}} - * - * @param values are constants of the same type - * apache/spark - */ - def isin(values: U*)(implicit e: CatalystIsin[U]): ThisType[T, Boolean] = - typed(self.untyped.isin(values:_*)) - - /** - * True if the current column is between the lower bound and upper bound, inclusive. - * - * @param lowerBound a constant of the same type - * @param upperBound a constant of the same type - * apache/spark - */ - def between(lowerBound: U, upperBound: U)(implicit i0: CatalystOrdered[U]): ThisType[T, Boolean] = - typed(self.untyped.between(lit(lowerBound)(self.uencoder).untyped, lit(upperBound)(self.uencoder).untyped)) - - /** - * True if the current column is between the lower bound and upper bound, inclusive. - * - * @param lowerBound another column of the same type - * @param upperBound another column of the same type - * apache/spark - */ - def between[TT1, TT2, W1, W2](lowerBound: ThisType[TT1, U], upperBound: ThisType[TT2, U]) - (implicit + * Returns true if the value of this column is contained in of the arguments. + * {{{ + * // The following selects people with age 15, 20, or 30. + * df.select( df('age).isin(15, 20, 30) ) + * }}} + * + * @param values are constants of the same type + * apache/spark + */ + def isin( + values: U* + )(implicit + e: CatalystIsin[U] + ): ThisType[T, Boolean] = + typed(self.untyped.isin(values: _*)) + + /** + * True if the current column is between the lower bound and upper bound, inclusive. + * + * @param lowerBound a constant of the same type + * @param upperBound a constant of the same type + * apache/spark + */ + def between( + lowerBound: U, + upperBound: U + )(implicit + i0: CatalystOrdered[U] + ): ThisType[T, Boolean] = + typed( + self.untyped.between( + lit(lowerBound)(self.uencoder).untyped, + lit(upperBound)(self.uencoder).untyped + ) + ) + + /** + * True if the current column is between the lower bound and upper bound, inclusive. + * + * @param lowerBound another column of the same type + * @param upperBound another column of the same type + * apache/spark + */ + def between[TT1, TT2, W1, W2]( + lowerBound: ThisType[TT1, U], + upperBound: ThisType[TT2, U] + )(implicit i0: CatalystOrdered[U], w0: FWith.Aux[T, TT1, W1], w1: FWith.Aux[TT2, W1, W2] ): ThisType[W2, Boolean] = - typed(self.untyped.between(lowerBound.untyped, upperBound.untyped)) + typed(self.untyped.between(lowerBound.untyped, upperBound.untyped)) /** - * Returns a nested column matching the field `symbol`. - * - * @param symbol the field symbol - * @tparam V the type of the nested field - */ - def field[V](symbol: Witness.Lt[Symbol])(implicit + * Returns a nested column matching the field `symbol`. + * + * @param symbol the field symbol + * @tparam V the type of the nested field + */ + def field[V]( + symbol: Witness.Lt[Symbol] + )(implicit i0: TypedColumn.Exists[U, symbol.T, V], i1: TypedEncoder[V] - ): ThisType[T, V] = + ): ThisType[T, V] = typed(self.untyped.getField(symbol.value.name)) } - -sealed class SortedTypedColumn[T, U](val expr: Expression)( - implicit - val uencoder: TypedEncoder[U] -) extends UntypedExpression[T] { - - def this(column: Column)(implicit e: TypedEncoder[U]) = { +sealed class SortedTypedColumn[T, U]( + val expr: Expression + )(implicit + val uencoder: TypedEncoder[U]) + extends UntypedExpression[T] { + + def this( + column: Column + )(implicit + e: TypedEncoder[U] + ) = { this(FramelessInternals.expr(column)) } @@ -899,16 +1281,24 @@ sealed class SortedTypedColumn[T, U](val expr: Expression)( } object SortedTypedColumn { - implicit def defaultAscending[T, U : CatalystOrdered](typedColumn: TypedColumn[T, U]): SortedTypedColumn[T, U] = + + implicit def defaultAscending[T, U: CatalystOrdered]( + typedColumn: TypedColumn[T, U] + ): SortedTypedColumn[T, U] = new SortedTypedColumn[T, U](typedColumn.untyped.asc)(typedColumn.uencoder) - object defaultAscendingPoly extends Poly1 { - implicit def caseTypedColumn[T, U : CatalystOrdered] = at[TypedColumn[T, U]](c => defaultAscending(c)) - implicit def caseTypeSortedColumn[T, U] = at[SortedTypedColumn[T, U]](identity) - } + object defaultAscendingPoly extends Poly1 { + + implicit def caseTypedColumn[T, U: CatalystOrdered] = + at[TypedColumn[T, U]](c => defaultAscending(c)) + + implicit def caseTypeSortedColumn[T, U] = + at[SortedTypedColumn[T, U]](identity) + } } object TypedColumn { + /** Evidence that type `T` has column `K` with type `V`. */ @implicitNotFound(msg = "No column ${K} of type ${V} in ${T}") trait Exists[T, K, V] @@ -917,37 +1307,46 @@ object TypedColumn { trait ExistsMany[T, K <: HList, V] object ExistsMany { - implicit def deriveCons[T, KH, KT <: HList, V0, V1] - (implicit + + implicit def deriveCons[T, KH, KT <: HList, V0, V1]( + implicit head: Exists[T, KH, V0], tail: ExistsMany[V0, KT, V1] ): ExistsMany[T, KH :: KT, V1] = - new ExistsMany[T, KH :: KT, V1] {} + new ExistsMany[T, KH :: KT, V1] {} - implicit def deriveHNil[T, K, V](implicit head: Exists[T, K, V]): ExistsMany[T, K :: HNil, V] = + implicit def deriveHNil[T, K, V]( + implicit + head: Exists[T, K, V] + ): ExistsMany[T, K :: HNil, V] = new ExistsMany[T, K :: HNil, V] {} } object Exists { - def apply[T, V](column: Witness)(implicit e: Exists[T, column.T, V]): Exists[T, column.T, V] = e - implicit def deriveRecord[T, H <: HList, K, V] - (implicit + def apply[T, V]( + column: Witness + )(implicit + e: Exists[T, column.T, V] + ): Exists[T, column.T, V] = e + + implicit def deriveRecord[T, H <: HList, K, V]( + implicit i0: LabelledGeneric.Aux[T, H], i1: Selector.Aux[H, K, V] ): Exists[T, K, V] = new Exists[T, K, V] {} } /** - * {{{ - * import frameless.TypedColumn - * - * case class Foo(id: Int, bar: String) - * - * val colbar: TypedColumn[Foo, String] = TypedColumn { foo: Foo => foo.bar } - * val colid = TypedColumn[Foo, Int](_.id) - * }}} - */ + * {{{ + * import frameless.TypedColumn + * + * case class Foo(id: Int, bar: String) + * + * val colbar: TypedColumn[Foo, String] = TypedColumn { foo: Foo => foo.bar } + * val colid = TypedColumn[Foo, Int](_.id) + * }}} + */ def apply[T, U](x: T => U): TypedColumn[T, U] = macro TypedColumnMacroImpl.applyImpl[T, U] diff --git a/dataset/src/main/scala/frameless/TypedColumnMacroImpl.scala b/dataset/src/main/scala/frameless/TypedColumnMacroImpl.scala index d741dccaa..23502ef3b 100644 --- a/dataset/src/main/scala/frameless/TypedColumnMacroImpl.scala +++ b/dataset/src/main/scala/frameless/TypedColumnMacroImpl.scala @@ -4,7 +4,10 @@ import scala.reflect.macros.whitebox private[frameless] object TypedColumnMacroImpl { - def applyImpl[T: c.WeakTypeTag, U: c.WeakTypeTag](c: whitebox.Context)(x: c.Tree): c.Expr[TypedColumn[T, U]] = { + def applyImpl[T: c.WeakTypeTag, U: c.WeakTypeTag]( + c: whitebox.Context + )(x: c.Tree + ): c.Expr[TypedColumn[T, U]] = { import c.universe._ val t = c.weakTypeOf[T] @@ -13,7 +16,9 @@ private[frameless] object TypedColumnMacroImpl { def buildExpression(path: List[String]): c.Expr[TypedColumn[T, U]] = { val columnName = path.mkString(".") - c.Expr[TypedColumn[T, U]](q"new _root_.frameless.TypedColumn[$t, $u](_root_.org.apache.spark.sql.FramelessInternals.expr(org.apache.spark.sql.functions.col($columnName)))") + c.Expr[TypedColumn[T, U]]( + q"new _root_.frameless.TypedColumn[$t, $u](_root_.org.apache.spark.sql.FramelessInternals.expr(org.apache.spark.sql.functions.col($columnName)))" + ) } def abort(msg: String) = c.abort(c.enclosingPosition, msg) @@ -48,34 +53,39 @@ private[frameless] object TypedColumnMacroImpl { } x match { - case fn: Function => fn.body match { - case select: Select if select.name.isTermName => - val expectedRoot: Option[String] = fn.vparams match { - case List(rt) if rt.rhs == EmptyTree => - Option.empty[String] - - case List(rt) => - Some(rt.toString) + case fn: Function => + fn.body match { + case select: Select if select.name.isTermName => + val expectedRoot: Option[String] = fn.vparams match { + case List(rt) if rt.rhs == EmptyTree => + Option.empty[String] + + case List(rt) => + Some(rt.toString) + + case u => + abort( + s"Select expression must have a single parameter: ${u mkString ", "}" + ) + } - case u => - abort(s"Select expression must have a single parameter: ${u mkString ", "}") - } + path(select, List.empty) match { + case root :: tail + if (expectedRoot.forall(_ == root) && check(t, tail)) => { + val colPath = tail.mkString(".") - path(select, List.empty) match { - case root :: tail if ( - expectedRoot.forall(_ == root) && check(t, tail)) => { - val colPath = tail.mkString(".") + c.Expr[TypedColumn[T, U]]( + q"new _root_.frameless.TypedColumn[$t, $u](_root_.org.apache.spark.sql.FramelessInternals.expr(org.apache.spark.sql.functions.col($colPath)))" + ) + } - c.Expr[TypedColumn[T, U]](q"new _root_.frameless.TypedColumn[$t, $u](_root_.org.apache.spark.sql.FramelessInternals.expr(org.apache.spark.sql.functions.col($colPath)))") + case _ => + abort(s"Invalid select expression: $select") } - case _ => - abort(s"Invalid select expression: $select") - } - - case t => - abort(s"Select expression expected: $t") - } + case t => + abort(s"Select expression expected: $t") + } case _ => abort(s"Function expected: $x") diff --git a/dataset/src/main/scala/frameless/TypedDataset.scala b/dataset/src/main/scala/frameless/TypedDataset.scala index 9946b457d..82a016a3a 100644 --- a/dataset/src/main/scala/frameless/TypedDataset.scala +++ b/dataset/src/main/scala/frameless/TypedDataset.scala @@ -4,36 +4,58 @@ import java.util import frameless.functions.CatalystExplodableCollection import frameless.ops._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Column, DataFrame, Dataset, FramelessInternals, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} +import org.apache.spark.sql.{ + Column, + DataFrame, + Dataset, + FramelessInternals, + SparkSession +} +import org.apache.spark.sql.catalyst.expressions.{ + Attribute, + AttributeReference, + Literal +} +import org.apache.spark.sql.catalyst.plans.logical.{ Join, JoinHint } import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.types.StructType import shapeless._ import shapeless.labelled.FieldType -import shapeless.ops.hlist.{Diff, IsHCons, Mapper, Prepend, ToTraversable, Tupler} -import shapeless.ops.record.{Keys, Modifier, Remover, Values} +import shapeless.ops.hlist.{ + Diff, + IsHCons, + Mapper, + Prepend, + ToTraversable, + Tupler +} +import shapeless.ops.record.{ Keys, Modifier, Remover, Values } import scala.language.experimental.macros -/** [[TypedDataset]] is a safer interface for working with `Dataset`. - * - * NOTE: Prefer `TypedDataset.create` over `new TypedDataset` unless you - * know what you are doing. - * - * Documentation marked "apache/spark" is thanks to apache/spark Contributors - * at https://github.com/apache/spark, licensed under Apache v2.0 available at - * http://www.apache.org/licenses/LICENSE-2.0 - */ -class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val encoder: TypedEncoder[T]) +/** + * [[TypedDataset]] is a safer interface for working with `Dataset`. + * + * NOTE: Prefer `TypedDataset.create` over `new TypedDataset` unless you + * know what you are doing. + * + * Documentation marked "apache/spark" is thanks to apache/spark Contributors + * at https://github.com/apache/spark, licensed under Apache v2.0 available at + * http://www.apache.org/licenses/LICENSE-2.0 + */ +class TypedDataset[T] protected[frameless] ( + val dataset: Dataset[T] + )(implicit + val encoder: TypedEncoder[T]) extends TypedDatasetForwarded[T] { self => private implicit val spark: SparkSession = dataset.sparkSession - /** Aggregates on the entire Dataset without groups. - * - * apache/spark - */ + /** + * Aggregates on the entire Dataset without groups. + * + * apache/spark + */ def agg[A](ca: TypedAggregate[T, A]): TypedDataset[A] = { implicit val ea = ca.uencoder val tuple1: TypedDataset[Tuple1[A]] = aggMany(ca) @@ -42,10 +64,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val TypedEncoder[A].catalystRepr match { case StructType(_) => // if column is struct, we use all its fields - val df = tuple1 - .dataset - .selectExpr("_1.*") - .as[A](TypedExpressionEncoder[A]) + val df = + tuple1.dataset.selectExpr("_1.*").as[A](TypedExpressionEncoder[A]) TypedDataset.create(df) case other => @@ -54,52 +74,59 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } } - /** Aggregates on the entire Dataset without groups. - * - * apache/spark - */ + /** + * Aggregates on the entire Dataset without groups. + * + * apache/spark + */ def agg[A, B]( - ca: TypedAggregate[T, A], - cb: TypedAggregate[T, B] - ): TypedDataset[(A, B)] = { + ca: TypedAggregate[T, A], + cb: TypedAggregate[T, B] + ): TypedDataset[(A, B)] = { implicit val (ea, eb) = (ca.uencoder, cb.uencoder) aggMany(ca, cb) } - /** Aggregates on the entire Dataset without groups. - * - * apache/spark - */ + /** + * Aggregates on the entire Dataset without groups. + * + * apache/spark + */ def agg[A, B, C]( - ca: TypedAggregate[T, A], - cb: TypedAggregate[T, B], - cc: TypedAggregate[T, C] - ): TypedDataset[(A, B, C)] = { + ca: TypedAggregate[T, A], + cb: TypedAggregate[T, B], + cc: TypedAggregate[T, C] + ): TypedDataset[(A, B, C)] = { implicit val (ea, eb, ec) = (ca.uencoder, cb.uencoder, cc.uencoder) aggMany(ca, cb, cc) } - /** Aggregates on the entire Dataset without groups. - * - * apache/spark - */ + /** + * Aggregates on the entire Dataset without groups. + * + * apache/spark + */ def agg[A, B, C, D]( - ca: TypedAggregate[T, A], - cb: TypedAggregate[T, B], - cc: TypedAggregate[T, C], - cd: TypedAggregate[T, D] - ): TypedDataset[(A, B, C, D)] = { - implicit val (ea, eb, ec, ed) = (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder) + ca: TypedAggregate[T, A], + cb: TypedAggregate[T, B], + cc: TypedAggregate[T, C], + cd: TypedAggregate[T, D] + ): TypedDataset[(A, B, C, D)] = { + implicit val (ea, eb, ec, ed) = + (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder) aggMany(ca, cb, cc, cd) } - /** Aggregates on the entire Dataset without groups. - * - * apache/spark - */ + /** + * Aggregates on the entire Dataset without groups. + * + * apache/spark + */ object aggMany extends ProductArgs { - def applyProduct[U <: HList, Out0 <: HList, Out](columns: U) - (implicit + + def applyProduct[U <: HList, Out0 <: HList, Out]( + columns: U + )(implicit i0: AggregateTypes.Aux[T, U, Out0], i1: ToTraversable.Aux[U, List, UntypedExpression[T]], i2: Tupler.Aux[Out0, Out], @@ -109,7 +136,7 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val val underlyingColumns = columns.toList[UntypedExpression[T]] val cols: Seq[Column] = for { (c, i) <- columns.toList[UntypedExpression[T]].zipWithIndex - } yield FramelessInternals.column(c.expr).as(s"_${i+1}") + } yield FramelessInternals.column(c.expr).as(s"_${i + 1}") // Workaround to SPARK-20346. One alternative is to allow the result to be Vector(null) for empty DataFrames. // Another one would be to return an Option. @@ -117,129 +144,163 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val for { (c, i) <- underlyingColumns.zipWithIndex if !c.uencoder.nullable - } yield s"_${i+1} is not null" - ).mkString(" or ") + } yield s"_${i + 1} is not null" + ).mkString(" or ") - val selected = dataset.toDF().agg(cols.head, cols.tail:_*).as[Out](TypedExpressionEncoder[Out]) - TypedDataset.create[Out](if (filterStr.isEmpty) selected else selected.filter(filterStr)) + val selected = dataset + .toDF() + .agg(cols.head, cols.tail: _*) + .as[Out](TypedExpressionEncoder[Out]) + TypedDataset.create[Out]( + if (filterStr.isEmpty) selected else selected.filter(filterStr) + ) } } /** Returns a new [[TypedDataset]] where each record has been mapped on to the specified type. */ - def as[U]()(implicit as: As[T, U]): TypedDataset[U] = { + def as[U]( + )(implicit + as: As[T, U] + ): TypedDataset[U] = { implicit val uencoder = as.encoder TypedDataset.create(dataset.as[U](TypedExpressionEncoder[U])) } - /** Returns a checkpointed version of this [[TypedDataset]]. Checkpointing can be used to truncate the - * logical plan of this Dataset, which is especially useful in iterative algorithms where the - * plan may grow exponentially. It will be saved to files inside the checkpoint - * directory set with `SparkContext#setCheckpointDir`. - * - * Differs from `Dataset#checkpoint` by wrapping its result into an effect-suspending `F[_]`. - * - * apache/spark - */ - def checkpoint[F[_]](eager: Boolean)(implicit F: SparkDelay[F]): F[TypedDataset[T]] = + /** + * Returns a checkpointed version of this [[TypedDataset]]. Checkpointing can be used to truncate the + * logical plan of this Dataset, which is especially useful in iterative algorithms where the + * plan may grow exponentially. It will be saved to files inside the checkpoint + * directory set with `SparkContext#setCheckpointDir`. + * + * Differs from `Dataset#checkpoint` by wrapping its result into an effect-suspending `F[_]`. + * + * apache/spark + */ + def checkpoint[F[_]]( + eager: Boolean + )(implicit + F: SparkDelay[F] + ): F[TypedDataset[T]] = F.delay(TypedDataset.create[T](dataset.checkpoint(eager))) - /** Returns a new [[TypedDataset]] where each record has been mapped on to the specified type. - * Unlike `as` the projection U may include a subset of the columns of T and the column names and types must agree. - * - * {{{ - * case class Foo(i: Int, j: String) - * case class Bar(j: String) - * - * val t: TypedDataset[Foo] = ... - * val b: TypedDataset[Bar] = t.project[Bar] - * - * case class BarErr(e: String) - * // The following does not compile because `Foo` doesn't have a field with name `e` - * val e: TypedDataset[BarErr] = t.project[BarErr] - * }}} - */ - def project[U](implicit projector: SmartProject[T,U]): TypedDataset[U] = projector.apply(this) - - /** Returns a new [[TypedDataset]] that contains the elements of both this and the `other` [[TypedDataset]] - * combined. - * - * Note that, this function is not a typical set union operation, in that it does not eliminate - * duplicate items. As such, it is analogous to `UNION ALL` in SQL. - * - * Differs from `Dataset#union` by aligning fields if possible. - * It will not compile if `Datasets` have not compatible schema. - * - * Example: - * {{{ - * case class Foo(x: Int, y: Long) - * case class Bar(y: Long, x: Int) - * case class Faz(x: Int, y: Int, z: Int) - * - * foo: TypedDataset[Foo] = ... - * bar: TypedDataset[Bar] = ... - * faz: TypedDataset[Faz] = ... - * - * foo union bar: TypedDataset[Foo] - * foo union faz: TypedDataset[Foo] - * // won't compile, you need to reverse order, you can't project from less fields to more - * faz union foo - * - * }}} - * - * apache/spark - */ - def union[U: TypedEncoder](other: TypedDataset[U])(implicit projector: SmartProject[U, T]): TypedDataset[T] = + /** + * Returns a new [[TypedDataset]] where each record has been mapped on to the specified type. + * Unlike `as` the projection U may include a subset of the columns of T and the column names and types must agree. + * + * {{{ + * case class Foo(i: Int, j: String) + * case class Bar(j: String) + * + * val t: TypedDataset[Foo] = ... + * val b: TypedDataset[Bar] = t.project[Bar] + * + * case class BarErr(e: String) + * // The following does not compile because `Foo` doesn't have a field with name `e` + * val e: TypedDataset[BarErr] = t.project[BarErr] + * }}} + */ + def project[U]( + implicit + projector: SmartProject[T, U] + ): TypedDataset[U] = projector.apply(this) + + /** + * Returns a new [[TypedDataset]] that contains the elements of both this and the `other` [[TypedDataset]] + * combined. + * + * Note that, this function is not a typical set union operation, in that it does not eliminate + * duplicate items. As such, it is analogous to `UNION ALL` in SQL. + * + * Differs from `Dataset#union` by aligning fields if possible. + * It will not compile if `Datasets` have not compatible schema. + * + * Example: + * {{{ + * case class Foo(x: Int, y: Long) + * case class Bar(y: Long, x: Int) + * case class Faz(x: Int, y: Int, z: Int) + * + * foo: TypedDataset[Foo] = ... + * bar: TypedDataset[Bar] = ... + * faz: TypedDataset[Faz] = ... + * + * foo union bar: TypedDataset[Foo] + * foo union faz: TypedDataset[Foo] + * // won't compile, you need to reverse order, you can't project from less fields to more + * faz union foo + * + * }}} + * + * apache/spark + */ + def union[U: TypedEncoder]( + other: TypedDataset[U] + )(implicit + projector: SmartProject[U, T] + ): TypedDataset[T] = TypedDataset.create(dataset.union(other.project[T].dataset)) - /** Returns a new [[TypedDataset]] that contains the elements of both this and the `other` [[TypedDataset]] - * combined. - * - * Note that, this function is not a typical set union operation, in that it does not eliminate - * duplicate items. As such, it is analogous to `UNION ALL` in SQL. - * - * apache/spark - */ + /** + * Returns a new [[TypedDataset]] that contains the elements of both this and the `other` [[TypedDataset]] + * combined. + * + * Note that, this function is not a typical set union operation, in that it does not eliminate + * duplicate items. As such, it is analogous to `UNION ALL` in SQL. + * + * apache/spark + */ def union(other: TypedDataset[T]): TypedDataset[T] = { TypedDataset.create(dataset.union(other.dataset)) } - /** Returns the number of elements in the [[TypedDataset]]. - * - * Differs from `Dataset#count` by wrapping its result into an effect-suspending `F[_]`. - */ - def count[F[_]]()(implicit F: SparkDelay[F]): F[Long] = + /** + * Returns the number of elements in the [[TypedDataset]]. + * + * Differs from `Dataset#count` by wrapping its result into an effect-suspending `F[_]`. + */ + def count[F[_]]( + )(implicit + F: SparkDelay[F] + ): F[Long] = F.delay(dataset.count()) - /** Returns `TypedColumn` of type `A` given its name (alias for `col`). - * - * {{{ - * tf('id) - * }}} - * - * It is statically checked that column with such name exists and has type `A`. - */ - def apply[A](column: Witness.Lt[Symbol]) - (implicit + /** + * Returns `TypedColumn` of type `A` given its name (alias for `col`). + * + * {{{ + * tf('id) + * }}} + * + * It is statically checked that column with such name exists and has type `A`. + */ + def apply[A]( + column: Witness.Lt[Symbol] + )(implicit i0: TypedColumn.Exists[T, column.T, A], i1: TypedEncoder[A] ): TypedColumn[T, A] = col(column) - /** Returns `TypedColumn` of type `A` given its name. - * - * {{{ - * tf.col('id) - * }}} - * - * It is statically checked that column with such name exists and has type `A`. - */ - def col[A](column: Witness.Lt[Symbol]) - (implicit + /** + * Returns `TypedColumn` of type `A` given its name. + * + * {{{ + * tf.col('id) + * }}} + * + * It is statically checked that column with such name exists and has type `A`. + */ + def col[A]( + column: Witness.Lt[Symbol] + )(implicit i0: TypedColumn.Exists[T, column.T, A], i1: TypedEncoder[A] ): TypedColumn[T, A] = - new TypedColumn[T, A](dataset(column.value.name).as[A](TypedExpressionEncoder[A])) + new TypedColumn[T, A]( + dataset(column.value.name).as[A](TypedExpressionEncoder[A]) + ) - /** Returns `TypedColumn` of type `A` given a lambda indicating the field. + /** + * Returns `TypedColumn` of type `A` given a lambda indicating the field. * * {{{ * td.col(_.id) @@ -250,12 +311,13 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val def col[A](x: Function1[T, A]): TypedColumn[T, A] = macro TypedColumnMacroImpl.applyImpl[T, A] - /** Projects the entire `TypedDataset[T]` into a single column of type `TypedColumn[T,T]`. - * {{{ - * ts: TypedDataset[Foo] = ... - * ts.select(ts.asCol, ts.asCol): TypedDataset[(Foo,Foo)] - * }}} - */ + /** + * Projects the entire `TypedDataset[T]` into a single column of type `TypedColumn[T,T]`. + * {{{ + * ts: TypedDataset[Foo] = ... + * ts.select(ts.asCol, ts.asCol): TypedDataset[(Foo,Foo)] + * }}} + */ def asCol: TypedColumn[T, T] = { val projectedColumn: Column = encoder.catalystRepr match { case StructType(_) => @@ -265,78 +327,98 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val case _ => dataset.col(dataset.columns.head) } - - new TypedColumn[T,T](projectedColumn) + + new TypedColumn[T, T](projectedColumn) } - /** References the entire `TypedDataset[T]` as a single column - * of type `TypedColumn[T,T]` so it can be used in a join operation. - * - * {{{ - * def nameJoin(ds1: TypedDataset[Person], ds2: TypedDataset[Name]) = - * ds1.joinLeftSemi(ds2)(ds1.col('name) === ds2.asJoinColValue) - * }}} - */ - def asJoinColValue(implicit i0: IsValueClass[T]): TypedColumn[T, T] = { + /** + * References the entire `TypedDataset[T]` as a single column + * of type `TypedColumn[T,T]` so it can be used in a join operation. + * + * {{{ + * def nameJoin(ds1: TypedDataset[Person], ds2: TypedDataset[Name]) = + * ds1.joinLeftSemi(ds2)(ds1.col('name) === ds2.asJoinColValue) + * }}} + */ + def asJoinColValue( + implicit + i0: IsValueClass[T] + ): TypedColumn[T, T] = { import _root_.frameless.syntax._ dataset.col("value").typedColumn } object colMany extends SingletonProductArgs { - def applyProduct[U <: HList, Out](columns: U) - (implicit + + def applyProduct[U <: HList, Out]( + columns: U + )(implicit i0: TypedColumn.ExistsMany[T, U, Out], i1: TypedEncoder[Out], i2: ToTraversable.Aux[U, List, Symbol] ): TypedColumn[T, Out] = { - val names = columns.toList[Symbol].map(_.name) - val colExpr = FramelessInternals.resolveExpr(dataset, names) - new TypedColumn[T, Out](colExpr) - } + val names = columns.toList[Symbol].map(_.name) + val colExpr = FramelessInternals.resolveExpr(dataset, names) + new TypedColumn[T, Out](colExpr) + } } - /** Right hand side disambiguation of `col` for join expressions. - * To be used when writting self-joins, noop in other circumstances. - * - * Note: In vanilla Spark, disambiguation in self-joins is acheaved using - * String based aliases, which is obviously unsafe. - */ - def colRight[A](column: Witness.Lt[Symbol]) - (implicit + /** + * Right hand side disambiguation of `col` for join expressions. + * To be used when writting self-joins, noop in other circumstances. + * + * Note: In vanilla Spark, disambiguation in self-joins is acheaved using + * String based aliases, which is obviously unsafe. + */ + def colRight[A]( + column: Witness.Lt[Symbol] + )(implicit i0: TypedColumn.Exists[T, column.T, A], i1: TypedEncoder[A] ): TypedColumn[T, A] = - new TypedColumn[T, A](FramelessInternals.DisambiguateRight(col(column).expr)) - - /** Left hand side disambiguation of `col` for join expressions. - * To be used when writting self-joins, noop in other circumstances. - * - * Note: In vanilla Spark, disambiguation in self-joins is acheaved using - * String based aliases, which is obviously unsafe. - */ - def colLeft[A](column: Witness.Lt[Symbol]) - (implicit + new TypedColumn[T, A]( + FramelessInternals.DisambiguateRight(col(column).expr) + ) + + /** + * Left hand side disambiguation of `col` for join expressions. + * To be used when writting self-joins, noop in other circumstances. + * + * Note: In vanilla Spark, disambiguation in self-joins is acheaved using + * String based aliases, which is obviously unsafe. + */ + def colLeft[A]( + column: Witness.Lt[Symbol] + )(implicit i0: TypedColumn.Exists[T, column.T, A], i1: TypedEncoder[A] ): TypedColumn[T, A] = - new TypedColumn[T, A](FramelessInternals.DisambiguateLeft(col(column).expr)) - - /** Returns a `Seq` that contains all the elements in this [[TypedDataset]]. - * - * Running this operation requires moving all the data into the application's driver process, and - * doing so on a very large [[TypedDataset]] can crash the driver process with OutOfMemoryError. - * - * Differs from `Dataset#collect` by wrapping its result into an effect-suspending `F[_]`. - */ - def collect[F[_]]()(implicit F: SparkDelay[F]): F[Seq[T]] = + new TypedColumn[T, A](FramelessInternals.DisambiguateLeft(col(column).expr)) + + /** + * Returns a `Seq` that contains all the elements in this [[TypedDataset]]. + * + * Running this operation requires moving all the data into the application's driver process, and + * doing so on a very large [[TypedDataset]] can crash the driver process with OutOfMemoryError. + * + * Differs from `Dataset#collect` by wrapping its result into an effect-suspending `F[_]`. + */ + def collect[F[_]]( + )(implicit + F: SparkDelay[F] + ): F[Seq[T]] = F.delay(dataset.collect().toSeq) - /** Optionally returns the first element in this [[TypedDataset]]. - * - * Differs from `Dataset#first` by wrapping its result into an `Option` and an effect-suspending `F[_]`. - */ - def firstOption[F[_]]()(implicit F: SparkDelay[F]): F[Option[T]] = + /** + * Optionally returns the first element in this [[TypedDataset]]. + * + * Differs from `Dataset#first` by wrapping its result into an `Option` and an effect-suspending `F[_]`. + */ + def firstOption[F[_]]( + )(implicit + F: SparkDelay[F] + ): F[Option[T]] = F.delay { try { Option(dataset.first()) @@ -345,355 +427,468 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } } - /** Returns the first `num` elements of this [[TypedDataset]] as a `Seq`. - * - * Running take requires moving data into the application's driver process, and doing so with - * a very large `num` can crash the driver process with OutOfMemoryError. - * - * Differs from `Dataset#take` by wrapping its result into an effect-suspending `F[_]`. - * - * apache/spark - */ - def take[F[_]](num: Int)(implicit F: SparkDelay[F]): F[Seq[T]] = + /** + * Returns the first `num` elements of this [[TypedDataset]] as a `Seq`. + * + * Running take requires moving data into the application's driver process, and doing so with + * a very large `num` can crash the driver process with OutOfMemoryError. + * + * Differs from `Dataset#take` by wrapping its result into an effect-suspending `F[_]`. + * + * apache/spark + */ + def take[F[_]]( + num: Int + )(implicit + F: SparkDelay[F] + ): F[Seq[T]] = F.delay(dataset.take(num).toSeq) - /** Return an iterator that contains all rows in this [[TypedDataset]]. - * - * The iterator will consume as much memory as the largest partition in this [[TypedDataset]]. - * - * NOTE: this results in multiple Spark jobs, and if the input [[TypedDataset]] is the result - * of a wide transformation (e.g. join with different partitioners), to avoid - * recomputing the input [[TypedDataset]] should be cached first. - * - * Differs from `Dataset#toLocalIterator()` by wrapping its result into an effect-suspending `F[_]`. - * - * apache/spark - */ - def toLocalIterator[F[_]]()(implicit F: SparkDelay[F]): F[util.Iterator[T]] = + /** + * Return an iterator that contains all rows in this [[TypedDataset]]. + * + * The iterator will consume as much memory as the largest partition in this [[TypedDataset]]. + * + * NOTE: this results in multiple Spark jobs, and if the input [[TypedDataset]] is the result + * of a wide transformation (e.g. join with different partitioners), to avoid + * recomputing the input [[TypedDataset]] should be cached first. + * + * Differs from `Dataset#toLocalIterator()` by wrapping its result into an effect-suspending `F[_]`. + * + * apache/spark + */ + def toLocalIterator[F[_]]( + )(implicit + F: SparkDelay[F] + ): F[util.Iterator[T]] = F.delay(dataset.toLocalIterator()) - /** Alias for firstOption(). - */ - def headOption[F[_]]()(implicit F: SparkDelay[F]): F[Option[T]] = firstOption() + /** + * Alias for firstOption(). + */ + def headOption[F[_]]( + )(implicit + F: SparkDelay[F] + ): F[Option[T]] = firstOption() - /** Alias for take(). - */ - def head[F[_]](num: Int)(implicit F: SparkDelay[F]): F[Seq[T]] = take(num) + /** + * Alias for take(). + */ + def head[F[_]]( + num: Int + )(implicit + F: SparkDelay[F] + ): F[Seq[T]] = take(num) // $COVERAGE-OFF$ - /** Alias for firstOption(). - */ - @deprecated("Method may throw exception. Use headOption or firstOption instead.", "0.5.0") + /** + * Alias for firstOption(). + */ + @deprecated( + "Method may throw exception. Use headOption or firstOption instead.", + "0.5.0" + ) def head: T = dataset.head() - /** Alias for firstOption(). - */ - @deprecated("Method may throw exception. Use headOption or firstOption instead.", "0.5.0") + /** + * Alias for firstOption(). + */ + @deprecated( + "Method may throw exception. Use headOption or firstOption instead.", + "0.5.0" + ) def first: T = dataset.head() // $COVERAGE-ONN$ - /** Displays the content of this [[TypedDataset]] in a tabular form. Strings more than 20 characters - * will be truncated, and all cells will be aligned right. For example: - * {{{ - * year month AVG('Adj Close) MAX('Adj Close) - * 1980 12 0.503218 0.595103 - * 1981 01 0.523289 0.570307 - * 1982 02 0.436504 0.475256 - * 1983 03 0.410516 0.442194 - * 1984 04 0.450090 0.483521 - * }}} - * @param numRows Number of rows to show - * @param truncate Whether truncate long strings. If true, strings more than 20 characters will - * be truncated and all cells will be aligned right - * - * Differs from `Dataset#show` by wrapping its result into an effect-suspending `F[_]`. - * - * apache/spark - */ - def show[F[_]](numRows: Int = 20, truncate: Boolean = true)(implicit F: SparkDelay[F]): F[Unit] = + /** + * Displays the content of this [[TypedDataset]] in a tabular form. Strings more than 20 characters + * will be truncated, and all cells will be aligned right. For example: + * {{{ + * year month AVG('Adj Close) MAX('Adj Close) + * 1980 12 0.503218 0.595103 + * 1981 01 0.523289 0.570307 + * 1982 02 0.436504 0.475256 + * 1983 03 0.410516 0.442194 + * 1984 04 0.450090 0.483521 + * }}} + * @param numRows Number of rows to show + * @param truncate Whether truncate long strings. If true, strings more than 20 characters will + * be truncated and all cells will be aligned right + * + * Differs from `Dataset#show` by wrapping its result into an effect-suspending `F[_]`. + * + * apache/spark + */ + def show[F[_]]( + numRows: Int = 20, + truncate: Boolean = true + )(implicit + F: SparkDelay[F] + ): F[Unit] = F.delay(dataset.show(numRows, truncate)) - /** Returns a new [[frameless.TypedDataset]] that only contains elements where `column` is `true`. - * - * Differs from `TypedDatasetForward#filter` by taking a `TypedColumn[T, Boolean]` instead of a - * `T => Boolean`. Using a column expression instead of a regular function save one Spark → Scala - * deserialization which leads to better performance. - */ + /** + * Returns a new [[frameless.TypedDataset]] that only contains elements where `column` is `true`. + * + * Differs from `TypedDatasetForward#filter` by taking a `TypedColumn[T, Boolean]` instead of a + * `T => Boolean`. Using a column expression instead of a regular function save one Spark → Scala + * deserialization which leads to better performance. + */ def filter(column: TypedColumn[T, Boolean]): TypedDataset[T] = { - val filtered = dataset.toDF() - .filter(column.untyped) - .as[T](TypedExpressionEncoder[T]) + val filtered = + dataset.toDF().filter(column.untyped).as[T](TypedExpressionEncoder[T]) TypedDataset.create[T](filtered) } - /** Runs `func` on each element of this [[TypedDataset]]. - * - * Differs from `Dataset#foreach` by wrapping its result into an effect-suspending `F[_]`. - */ - def foreach[F[_]](func: T => Unit)(implicit F: SparkDelay[F]): F[Unit] = + /** + * Runs `func` on each element of this [[TypedDataset]]. + * + * Differs from `Dataset#foreach` by wrapping its result into an effect-suspending `F[_]`. + */ + def foreach[F[_]]( + func: T => Unit + )(implicit + F: SparkDelay[F] + ): F[Unit] = F.delay(dataset.foreach(func)) - /** Runs `func` on each partition of this [[TypedDataset]]. - * - * Differs from `Dataset#foreachPartition` by wrapping its result into an effect-suspending `F[_]`. - */ - def foreachPartition[F[_]](func: Iterator[T] => Unit)(implicit F: SparkDelay[F]): F[Unit] = + /** + * Runs `func` on each partition of this [[TypedDataset]]. + * + * Differs from `Dataset#foreachPartition` by wrapping its result into an effect-suspending `F[_]`. + */ + def foreachPartition[F[_]]( + func: Iterator[T] => Unit + )(implicit + F: SparkDelay[F] + ): F[Unit] = F.delay(dataset.foreachPartition(func)) /** - * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified column, - * so we can run aggregation on it. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified column, + * so we can run aggregation on it. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ def cube[K1]( - c1: TypedColumn[T, K1] - ): Cube1Ops[K1, T] = new Cube1Ops[K1, T](this, c1) - - /** - * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified columns, - * so we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + c1: TypedColumn[T, K1] + ): Cube1Ops[K1, T] = new Cube1Ops[K1, T](this, c1) + + /** + * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified columns, + * so we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ def cube[K1, K2]( - c1: TypedColumn[T, K1], - c2: TypedColumn[T, K2] - ): Cube2Ops[K1, K2, T] = new Cube2Ops[K1, K2, T](this, c1, c2) - - /** - * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified columns, - * so we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * {{{ - * case class MyClass(a: Int, b: Int, c: Int) - * val ds: TypedDataset[MyClass] - - * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = - * ds.cubeMany(ds('a), ds('b)).agg(count[MyClass]()) - * - * // original dataset: - * a b c - * 10 20 1 - * 15 25 2 - * - * // after aggregation: - * _1 _2 _3 - * 15 null 1 - * 15 25 1 - * null null 2 - * null 25 1 - * null 20 1 - * 10 null 1 - * 10 20 1 - * - * }}} - * - * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + c1: TypedColumn[T, K1], + c2: TypedColumn[T, K2] + ): Cube2Ops[K1, K2, T] = new Cube2Ops[K1, K2, T](this, c1, c2) + + /** + * Create a multi-dimensional cube for the current [[TypedDataset]] using the specified columns, + * so we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * {{{ + * case class MyClass(a: Int, b: Int, c: Int) + * val ds: TypedDataset[MyClass] + * + * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = + * ds.cubeMany(ds('a), ds('b)).agg(count[MyClass]()) + * + * // original dataset: + * a b c + * 10 20 1 + * 15 25 2 + * + * // after aggregation: + * _1 _2 _3 + * 15 null 1 + * 15 25 1 + * null null 2 + * null 25 1 + * null 20 1 + * 10 null 1 + * 10 20 1 + * + * }}} + * + * Differs from `Dataset#cube` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ object cubeMany extends ProductArgs { - def applyProduct[TK <: HList, K <: HList, KT](groupedBy: TK) - (implicit + + def applyProduct[TK <: HList, K <: HList, KT]( + groupedBy: TK + )(implicit i0: ColumnTypes.Aux[T, TK, K], i1: Tupler.Aux[K, KT], i2: ToTraversable.Aux[TK, List, UntypedExpression[T]] - ): CubeManyOps[T, TK, K, KT] = new CubeManyOps[T, TK, K, KT](self, groupedBy) + ): CubeManyOps[T, TK, K, KT] = + new CubeManyOps[T, TK, K, KT](self, groupedBy) } /** - * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * apache/spark - */ + * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * apache/spark + */ def groupBy[K1]( - c1: TypedColumn[T, K1] - ): GroupedBy1Ops[K1, T] = new GroupedBy1Ops[K1, T](this, c1) + c1: TypedColumn[T, K1] + ): GroupedBy1Ops[K1, T] = new GroupedBy1Ops[K1, T](this, c1) /** - * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * apache/spark - */ + * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * apache/spark + */ def groupBy[K1, K2]( - c1: TypedColumn[T, K1], - c2: TypedColumn[T, K2] - ): GroupedBy2Ops[K1, K2, T] = new GroupedBy2Ops[K1, K2, T](this, c1, c2) - - /** - * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * {{{ - * case class MyClass(a: Int, b: Int, c: Int) - * val ds: TypedDataset[MyClass] - * - * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = - * ds.groupByMany(ds('a), ds('b)).agg(count[MyClass]()) - * - * // original dataset: - * a b c - * 10 20 1 - * 15 25 2 - * - * // after aggregation: - * _1 _2 _3 - * 10 20 1 - * 15 25 1 - * - * }}} - * - * apache/spark - */ + c1: TypedColumn[T, K1], + c2: TypedColumn[T, K2] + ): GroupedBy2Ops[K1, K2, T] = new GroupedBy2Ops[K1, K2, T](this, c1, c2) + + /** + * Groups the [[TypedDataset]] using the specified columns, so that we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * {{{ + * case class MyClass(a: Int, b: Int, c: Int) + * val ds: TypedDataset[MyClass] + * + * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = + * ds.groupByMany(ds('a), ds('b)).agg(count[MyClass]()) + * + * // original dataset: + * a b c + * 10 20 1 + * 15 25 2 + * + * // after aggregation: + * _1 _2 _3 + * 10 20 1 + * 15 25 1 + * + * }}} + * + * apache/spark + */ object groupByMany extends ProductArgs { - def applyProduct[TK <: HList, K <: HList, KT](groupedBy: TK) - (implicit + + def applyProduct[TK <: HList, K <: HList, KT]( + groupedBy: TK + )(implicit i0: ColumnTypes.Aux[T, TK, K], i1: Tupler.Aux[K, KT], i2: ToTraversable.Aux[TK, List, UntypedExpression[T]] - ): GroupedByManyOps[T, TK, K, KT] = new GroupedByManyOps[T, TK, K, KT](self, groupedBy) + ): GroupedByManyOps[T, TK, K, KT] = + new GroupedByManyOps[T, TK, K, KT](self, groupedBy) } /** - * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified column, - * so we can run aggregation on it. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified column, + * so we can run aggregation on it. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ def rollup[K1]( - c1: TypedColumn[T, K1] - ): Rollup1Ops[K1, T] = new Rollup1Ops[K1, T](this, c1) - - /** - * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified columns, - * so we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + c1: TypedColumn[T, K1] + ): Rollup1Ops[K1, T] = new Rollup1Ops[K1, T](this, c1) + + /** + * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified columns, + * so we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ def rollup[K1, K2]( - c1: TypedColumn[T, K1], - c2: TypedColumn[T, K2] - ): Rollup2Ops[K1, K2, T] = new Rollup2Ops[K1, K2, T](this, c1, c2) - - /** - * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified columns, - * so we can run aggregation on them. - * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. - * - * {{{ - * case class MyClass(a: Int, b: Int, c: Int) - * val ds: TypedDataset[MyClass] - * - * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = - * ds.rollupMany(ds('a), ds('b)).agg(count[MyClass]()) - * - * // original dataset: - * a b c - * 10 20 1 - * 15 25 2 - * - * // after aggregation: - * _1 _2 _3 - * 15 null 1 - * 15 25 1 - * null null 2 - * 10 null 1 - * 10 20 1 - * - * }}} - * - * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. - * - * apache/spark - */ + c1: TypedColumn[T, K1], + c2: TypedColumn[T, K2] + ): Rollup2Ops[K1, K2, T] = new Rollup2Ops[K1, K2, T](this, c1, c2) + + /** + * Create a multi-dimensional rollup for the current [[TypedDataset]] using the specified columns, + * so we can run aggregation on them. + * See [[frameless.functions.AggregateFunctions]] for all the available aggregate functions. + * + * {{{ + * case class MyClass(a: Int, b: Int, c: Int) + * val ds: TypedDataset[MyClass] + * + * val cubeDataset: TypedDataset[(Option[A], Option[B], Long)] = + * ds.rollupMany(ds('a), ds('b)).agg(count[MyClass]()) + * + * // original dataset: + * a b c + * 10 20 1 + * 15 25 2 + * + * // after aggregation: + * _1 _2 _3 + * 15 null 1 + * 15 25 1 + * null null 2 + * 10 null 1 + * 10 20 1 + * + * }}} + * + * Differs from `Dataset#rollup` by wrapping values into `Option` instead of returning `null`. + * + * apache/spark + */ object rollupMany extends ProductArgs { - def applyProduct[TK <: HList, K <: HList, KT](groupedBy: TK) - (implicit + + def applyProduct[TK <: HList, K <: HList, KT]( + groupedBy: TK + )(implicit i0: ColumnTypes.Aux[T, TK, K], i1: Tupler.Aux[K, KT], i2: ToTraversable.Aux[TK, List, UntypedExpression[T]] - ): RollupManyOps[T, TK, K, KT] = new RollupManyOps[T, TK, K, KT](self, groupedBy) + ): RollupManyOps[T, TK, K, KT] = + new RollupManyOps[T, TK, K, KT](self, groupedBy) } /** Computes the cartesian project of `this` `Dataset` with the `other` `Dataset` */ - def joinCross[U](other: TypedDataset[U]) - (implicit e: TypedEncoder[(T, U)]): TypedDataset[(T, U)] = - new TypedDataset(self.dataset.joinWith(other.dataset, FramelessInternals.column(Literal(true)), "cross") - .as[(T, U)](TypedExpressionEncoder[(T, U)])) - - /** Computes the full outer join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinFull[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]) - (implicit e: TypedEncoder[(Option[T], Option[U])]): TypedDataset[(Option[T], Option[U])] = - new TypedDataset(self.dataset.joinWith(other.dataset, condition.untyped, "full") - .as[(Option[T], Option[U])](TypedExpressionEncoder[(Option[T], Option[U])])) - - /** Computes the inner join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinInner[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]) - (implicit e: TypedEncoder[(T, U)]): TypedDataset[(T, U)] = { - import FramelessInternals._ - - val leftPlan = logicalPlan(dataset) - val rightPlan = logicalPlan(other.dataset) - val join = disambiguate(Join(leftPlan, rightPlan, Inner, Some(condition.expr), JoinHint.NONE)) - val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) - val joinedDs = mkDataset(dataset, joinedPlan, TypedExpressionEncoder[(T, U)]) - - TypedDataset.create[(T, U)](joinedDs) - } + def joinCross[U]( + other: TypedDataset[U] + )(implicit + e: TypedEncoder[(T, U)] + ): TypedDataset[(T, U)] = + new TypedDataset( + self.dataset + .joinWith( + other.dataset, + FramelessInternals.column(Literal(true)), + "cross" + ) + .as[(T, U)](TypedExpressionEncoder[(T, U)]) + ) - /** Computes the left outer join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinLeft[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]) - (implicit e: TypedEncoder[(T, Option[U])]): TypedDataset[(T, Option[U])] = - new TypedDataset(self.dataset.joinWith(other.dataset, condition.untyped, "left_outer") - .as[(T, Option[U])](TypedExpressionEncoder[(T, Option[U])])) - - /** Computes the left semi join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinLeftSemi[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]): TypedDataset[T] = - new TypedDataset(self.dataset.join(other.dataset, condition.untyped, "leftsemi") - .as[T](TypedExpressionEncoder(encoder))) - - /** Computes the left anti join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinLeftAnti[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]): TypedDataset[T] = - new TypedDataset(self.dataset.join(other.dataset, condition.untyped, "leftanti") - .as[T](TypedExpressionEncoder(encoder))) - - /** Computes the right outer join of `this` `Dataset` with the `other` `Dataset`, - * returning a `Tuple2` for each pair where condition evaluates to true. - */ - def joinRight[U](other: TypedDataset[U])(condition: TypedColumn[T with U, Boolean]) - (implicit e: TypedEncoder[(Option[T], U)]): TypedDataset[(Option[T], U)] = - new TypedDataset(self.dataset.joinWith(other.dataset, condition.untyped, "right_outer") - .as[(Option[T], U)](TypedExpressionEncoder[(Option[T], U)])) + /** + * Computes the full outer join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinFull[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + )(implicit + e: TypedEncoder[(Option[T], Option[U])] + ): TypedDataset[(Option[T], Option[U])] = + new TypedDataset( + self.dataset + .joinWith(other.dataset, condition.untyped, "full") + .as[(Option[T], Option[U])]( + TypedExpressionEncoder[(Option[T], Option[U])] + ) + ) + + /** + * Computes the inner join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinInner[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + )(implicit + e: TypedEncoder[(T, U)] + ): TypedDataset[(T, U)] = { + import FramelessInternals._ + + val leftPlan = logicalPlan(dataset) + val rightPlan = logicalPlan(other.dataset) + val join = disambiguate( + Join(leftPlan, rightPlan, Inner, Some(condition.expr), JoinHint.NONE) + ) + val joinedPlan = joinPlan(dataset, join, leftPlan, rightPlan) + val joinedDs = + mkDataset(dataset, joinedPlan, TypedExpressionEncoder[(T, U)]) + + TypedDataset.create[(T, U)](joinedDs) + } + + /** + * Computes the left outer join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinLeft[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + )(implicit + e: TypedEncoder[(T, Option[U])] + ): TypedDataset[(T, Option[U])] = + new TypedDataset( + self.dataset + .joinWith(other.dataset, condition.untyped, "left_outer") + .as[(T, Option[U])](TypedExpressionEncoder[(T, Option[U])]) + ) + + /** + * Computes the left semi join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinLeftSemi[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + ): TypedDataset[T] = + new TypedDataset( + self.dataset + .join(other.dataset, condition.untyped, "leftsemi") + .as[T](TypedExpressionEncoder(encoder)) + ) + + /** + * Computes the left anti join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinLeftAnti[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + ): TypedDataset[T] = + new TypedDataset( + self.dataset + .join(other.dataset, condition.untyped, "leftanti") + .as[T](TypedExpressionEncoder(encoder)) + ) + + /** + * Computes the right outer join of `this` `Dataset` with the `other` `Dataset`, + * returning a `Tuple2` for each pair where condition evaluates to true. + */ + def joinRight[U]( + other: TypedDataset[U] + )(condition: TypedColumn[T with U, Boolean] + )(implicit + e: TypedEncoder[(Option[T], U)] + ): TypedDataset[(Option[T], U)] = + new TypedDataset( + self.dataset + .joinWith(other.dataset, condition.untyped, "right_outer") + .as[(Option[T], U)](TypedExpressionEncoder[(Option[T], U)]) + ) private def disambiguate(join: Join): Join = { - val plan = FramelessInternals.ofRows(dataset.sparkSession, join).queryExecution.analyzed.asInstanceOf[Join] + val plan = FramelessInternals + .ofRows(dataset.sparkSession, join) + .queryExecution + .analyzed + .asInstanceOf[Join] val disambiguated = plan.condition.map(_.transform { case FramelessInternals.DisambiguateLeft(tagged: AttributeReference) => val leftDs = FramelessInternals.ofRows(spark, plan.left) @@ -708,43 +903,82 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val plan.copy(condition = disambiguated) } - /** Takes a function from A => R and converts it to a UDF for TypedColumn[T, A] => TypedColumn[T, R]. - */ - def makeUDF[A: TypedEncoder, R: TypedEncoder](f: A => R): - TypedColumn[T, A] => TypedColumn[T, R] = functions.udf(f) - - /** Takes a function from (A1, A2) => R and converts it to a UDF for - * (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R]. - */ - def makeUDF[A1: TypedEncoder, A2: TypedEncoder, R: TypedEncoder](f: (A1, A2) => R): - (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R] = functions.udf(f) - - /** Takes a function from (A1, A2, A3) => R and converts it to a UDF for - * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3]) => TypedColumn[T, R]. - */ - def makeUDF[A1: TypedEncoder, A2: TypedEncoder, A3: TypedEncoder, R: TypedEncoder](f: (A1, A2, A3) => R): - (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3]) => TypedColumn[T, R] = functions.udf(f) - - /** Takes a function from (A1, A2, A3, A4) => R and converts it to a UDF for - * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4]) => TypedColumn[T, R]. - */ - def makeUDF[A1: TypedEncoder, A2: TypedEncoder, A3: TypedEncoder, A4: TypedEncoder, R: TypedEncoder](f: (A1, A2, A3, A4) => R): - (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4]) => TypedColumn[T, R] = functions.udf(f) - - /** Takes a function from (A1, A2, A3, A4, A5) => R and converts it to a UDF for - * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4], TypedColumn[T, A5]) => TypedColumn[T, R]. - */ - def makeUDF[A1: TypedEncoder, A2: TypedEncoder, A3: TypedEncoder, A4: TypedEncoder, A5: TypedEncoder, R: TypedEncoder](f: (A1, A2, A3, A4, A5) => R): - (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4], TypedColumn[T, A5]) => TypedColumn[T, R] = functions.udf(f) - - /** Type-safe projection from type T to Tuple1[A] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Takes a function from A => R and converts it to a UDF for TypedColumn[T, A] => TypedColumn[T, R]. + */ + def makeUDF[A: TypedEncoder, R: TypedEncoder]( + f: A => R + ): TypedColumn[T, A] => TypedColumn[T, R] = functions.udf(f) + + /** + * Takes a function from (A1, A2) => R and converts it to a UDF for + * (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R]. + */ + def makeUDF[A1: TypedEncoder, A2: TypedEncoder, R: TypedEncoder]( + f: (A1, A2) => R + ): (TypedColumn[T, A1], TypedColumn[T, A2]) => TypedColumn[T, R] = + functions.udf(f) + + /** + * Takes a function from (A1, A2, A3) => R and converts it to a UDF for + * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3]) => TypedColumn[T, R]. + */ + def makeUDF[ + A1: TypedEncoder, + A2: TypedEncoder, + A3: TypedEncoder, + R: TypedEncoder + ](f: (A1, A2, A3) => R + ): (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3]) => TypedColumn[T, R] = + functions.udf(f) + + /** + * Takes a function from (A1, A2, A3, A4) => R and converts it to a UDF for + * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4]) => TypedColumn[T, R]. + */ + def makeUDF[ + A1: TypedEncoder, + A2: TypedEncoder, + A3: TypedEncoder, + A4: TypedEncoder, + R: TypedEncoder + ](f: (A1, A2, A3, A4) => R + ): ( + TypedColumn[T, A1], + TypedColumn[T, A2], + TypedColumn[T, A3], + TypedColumn[T, A4] + ) => TypedColumn[T, R] = functions.udf(f) + + /** + * Takes a function from (A1, A2, A3, A4, A5) => R and converts it to a UDF for + * (TypedColumn[T, A1], TypedColumn[T, A2], TypedColumn[T, A3], TypedColumn[T, A4], TypedColumn[T, A5]) => TypedColumn[T, R]. + */ + def makeUDF[ + A1: TypedEncoder, + A2: TypedEncoder, + A3: TypedEncoder, + A4: TypedEncoder, + A5: TypedEncoder, + R: TypedEncoder + ](f: (A1, A2, A3, A4, A5) => R + ): ( + TypedColumn[T, A1], + TypedColumn[T, A2], + TypedColumn[T, A3], + TypedColumn[T, A4], + TypedColumn[T, A5] + ) => TypedColumn[T, R] = functions.udf(f) + + /** + * Type-safe projection from type T to Tuple1[A] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A]( - ca: TypedColumn[T, A] - ): TypedDataset[A] = { + ca: TypedColumn[T, A] + ): TypedDataset[A] = { implicit val ea = ca.uencoder val tuple1: TypedDataset[Tuple1[A]] = selectMany(ca) @@ -754,10 +988,8 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val TypedEncoder[A].catalystRepr match { case StructType(_) => // if column is struct, we use all its fields - val df = tuple1 - .dataset - .selectExpr("_1.*") - .as[A](TypedExpressionEncoder[A]) + val df = + tuple1.dataset.selectExpr("_1.*").as[A](TypedExpressionEncoder[A]) TypedDataset.create(df) case other => @@ -766,217 +998,290 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } } - /** Type-safe projection from type T to Tuple2[A,B] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple2[A,B] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B] - ): TypedDataset[(A, B)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B] + ): TypedDataset[(A, B)] = { implicit val (ea, eb) = (ca.uencoder, cb.uencoder) selectMany(ca, cb) } - /** Type-safe projection from type T to Tuple3[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple3[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C] - ): TypedDataset[(A, B, C)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C] + ): TypedDataset[(A, B, C)] = { implicit val (ea, eb, ec) = (ca.uencoder, cb.uencoder, cc.uencoder) selectMany(ca, cb, cc) } - /** Type-safe projection from type T to Tuple4[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple4[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D] - ): TypedDataset[(A, B, C, D)] = { - implicit val (ea, eb, ec, ed) = (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder) + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D] + ): TypedDataset[(A, B, C, D)] = { + implicit val (ea, eb, ec, ed) = + (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder) selectMany(ca, cb, cc, cd) } - /** Type-safe projection from type T to Tuple5[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple5[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E] - ): TypedDataset[(A, B, C, D, E)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E] + ): TypedDataset[(A, B, C, D, E)] = { implicit val (ea, eb, ec, ed, ee) = (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder) selectMany(ca, cb, cc, cd, ce) } - /** Type-safe projection from type T to Tuple6[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple6[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E, F]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E], - cf: TypedColumn[T, F] - ): TypedDataset[(A, B, C, D, E, F)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E], + cf: TypedColumn[T, F] + ): TypedDataset[(A, B, C, D, E, F)] = { implicit val (ea, eb, ec, ed, ee, ef) = - (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder, cf.uencoder) + ( + ca.uencoder, + cb.uencoder, + cc.uencoder, + cd.uencoder, + ce.uencoder, + cf.uencoder + ) selectMany(ca, cb, cc, cd, ce, cf) } - /** Type-safe projection from type T to Tuple7[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple7[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E, F, G]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E], - cf: TypedColumn[T, F], - cg: TypedColumn[T, G] - ): TypedDataset[(A, B, C, D, E, F, G)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E], + cf: TypedColumn[T, F], + cg: TypedColumn[T, G] + ): TypedDataset[(A, B, C, D, E, F, G)] = { implicit val (ea, eb, ec, ed, ee, ef, eg) = - (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder, cf.uencoder, cg.uencoder) + ( + ca.uencoder, + cb.uencoder, + cc.uencoder, + cd.uencoder, + ce.uencoder, + cf.uencoder, + cg.uencoder + ) selectMany(ca, cb, cc, cd, ce, cf, cg) } - /** Type-safe projection from type T to Tuple8[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple8[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E, F, G, H]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E], - cf: TypedColumn[T, F], - cg: TypedColumn[T, G], - ch: TypedColumn[T, H] - ): TypedDataset[(A, B, C, D, E, F, G, H)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E], + cf: TypedColumn[T, F], + cg: TypedColumn[T, G], + ch: TypedColumn[T, H] + ): TypedDataset[(A, B, C, D, E, F, G, H)] = { implicit val (ea, eb, ec, ed, ee, ef, eg, eh) = - (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder, cf.uencoder, cg.uencoder, ch.uencoder) + ( + ca.uencoder, + cb.uencoder, + cc.uencoder, + cd.uencoder, + ce.uencoder, + cf.uencoder, + cg.uencoder, + ch.uencoder + ) selectMany(ca, cb, cc, cd, ce, cf, cg, ch) } - /** Type-safe projection from type T to Tuple9[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple9[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E, F, G, H, I]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E], - cf: TypedColumn[T, F], - cg: TypedColumn[T, G], - ch: TypedColumn[T, H], - ci: TypedColumn[T, I] - ): TypedDataset[(A, B, C, D, E, F, G, H, I)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E], + cf: TypedColumn[T, F], + cg: TypedColumn[T, G], + ch: TypedColumn[T, H], + ci: TypedColumn[T, I] + ): TypedDataset[(A, B, C, D, E, F, G, H, I)] = { implicit val (ea, eb, ec, ed, ee, ef, eg, eh, ei) = - (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder, cf.uencoder, cg.uencoder, ch.uencoder, ci.uencoder) + ( + ca.uencoder, + cb.uencoder, + cc.uencoder, + cd.uencoder, + ce.uencoder, + cf.uencoder, + cg.uencoder, + ch.uencoder, + ci.uencoder + ) selectMany(ca, cb, cc, cd, ce, cf, cg, ch, ci) } - /** Type-safe projection from type T to Tuple10[A,B,...] - * {{{ - * d.select( d('a), d('a)+d('b), ... ) - * }}} - */ + /** + * Type-safe projection from type T to Tuple10[A,B,...] + * {{{ + * d.select( d('a), d('a)+d('b), ... ) + * }}} + */ def select[A, B, C, D, E, F, G, H, I, J]( - ca: TypedColumn[T, A], - cb: TypedColumn[T, B], - cc: TypedColumn[T, C], - cd: TypedColumn[T, D], - ce: TypedColumn[T, E], - cf: TypedColumn[T, F], - cg: TypedColumn[T, G], - ch: TypedColumn[T, H], - ci: TypedColumn[T, I], - cj: TypedColumn[T, J] - ): TypedDataset[(A, B, C, D, E, F, G, H, I, J)] = { + ca: TypedColumn[T, A], + cb: TypedColumn[T, B], + cc: TypedColumn[T, C], + cd: TypedColumn[T, D], + ce: TypedColumn[T, E], + cf: TypedColumn[T, F], + cg: TypedColumn[T, G], + ch: TypedColumn[T, H], + ci: TypedColumn[T, I], + cj: TypedColumn[T, J] + ): TypedDataset[(A, B, C, D, E, F, G, H, I, J)] = { implicit val (ea, eb, ec, ed, ee, ef, eg, eh, ei, ej) = - (ca.uencoder, cb.uencoder, cc.uencoder, cd.uencoder, ce.uencoder, cf.uencoder, cg.uencoder, ch.uencoder, ci.uencoder, cj.uencoder) + ( + ca.uencoder, + cb.uencoder, + cc.uencoder, + cd.uencoder, + ce.uencoder, + cf.uencoder, + cg.uencoder, + ch.uencoder, + ci.uencoder, + cj.uencoder + ) selectMany(ca, cb, cc, cd, ce, cf, cg, ch, ci, cj) } object selectMany extends ProductArgs { - def applyProduct[U <: HList, Out0 <: HList, Out](columns: U) - (implicit + + def applyProduct[U <: HList, Out0 <: HList, Out]( + columns: U + )(implicit i0: ColumnTypes.Aux[T, U, Out0], i1: ToTraversable.Aux[U, List, UntypedExpression[T]], i2: Tupler.Aux[Out0, Out], i3: TypedEncoder[Out] ): TypedDataset[Out] = { - val base = dataset.toDF() - .select(columns.toList[UntypedExpression[T]].map(c => FramelessInternals.column(c.expr)):_*) - val selected = base.as[Out](TypedExpressionEncoder[Out]) + val base = dataset + .toDF() + .select( + columns + .toList[UntypedExpression[T]] + .map(c => FramelessInternals.column(c.expr)): _* + ) + val selected = base.as[Out](TypedExpressionEncoder[Out]) - TypedDataset.create[Out](selected) - } + TypedDataset.create[Out](selected) + } } /** Sort each partition in the dataset using the columns selected. */ - def sortWithinPartitions[A: CatalystOrdered](ca: SortedTypedColumn[T, A]): TypedDataset[T] = + def sortWithinPartitions[A: CatalystOrdered]( + ca: SortedTypedColumn[T, A] + ): TypedDataset[T] = sortWithinPartitionsMany(ca) /** Sort each partition in the dataset using the columns selected. */ def sortWithinPartitions[A: CatalystOrdered, B: CatalystOrdered]( - ca: SortedTypedColumn[T, A], - cb: SortedTypedColumn[T, B] - ): TypedDataset[T] = sortWithinPartitionsMany(ca, cb) + ca: SortedTypedColumn[T, A], + cb: SortedTypedColumn[T, B] + ): TypedDataset[T] = sortWithinPartitionsMany(ca, cb) /** Sort each partition in the dataset using the columns selected. */ - def sortWithinPartitions[A: CatalystOrdered, B: CatalystOrdered, C: CatalystOrdered]( - ca: SortedTypedColumn[T, A], - cb: SortedTypedColumn[T, B], - cc: SortedTypedColumn[T, C] - ): TypedDataset[T] = sortWithinPartitionsMany(ca, cb, cc) - - /** Sort each partition in the dataset by the given column expressions - * Default sort order is ascending. - * {{{ - * d.sortWithinPartitionsMany(d('a), d('b).desc, d('c).asc) - * }}} - */ + def sortWithinPartitions[ + A: CatalystOrdered, + B: CatalystOrdered, + C: CatalystOrdered + ](ca: SortedTypedColumn[T, A], + cb: SortedTypedColumn[T, B], + cc: SortedTypedColumn[T, C] + ): TypedDataset[T] = sortWithinPartitionsMany(ca, cb, cc) + + /** + * Sort each partition in the dataset by the given column expressions + * Default sort order is ascending. + * {{{ + * d.sortWithinPartitionsMany(d('a), d('b).desc, d('c).asc) + * }}} + */ object sortWithinPartitionsMany extends ProductArgs { - def applyProduct[U <: HList, O <: HList](columns: U) - (implicit + + def applyProduct[U <: HList, O <: HList]( + columns: U + )(implicit i0: Mapper.Aux[SortedTypedColumn.defaultAscendingPoly.type, U, O], i1: ToTraversable.Aux[O, List, SortedTypedColumn[T, _]] ): TypedDataset[T] = { - val sorted = dataset.toDF() - .sortWithinPartitions(i0(columns).toList[SortedTypedColumn[T, _]].map(_.untyped):_*) + val sorted = dataset + .toDF() + .sortWithinPartitions( + i0(columns).toList[SortedTypedColumn[T, _]].map(_.untyped): _* + ) .as[T](TypedExpressionEncoder[T]) TypedDataset.create[T](sorted) @@ -984,273 +1289,316 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } /** Orders the TypedDataset using the column selected. */ - def orderBy[A: CatalystOrdered](ca: SortedTypedColumn[T, A]): TypedDataset[T] = + def orderBy[A: CatalystOrdered]( + ca: SortedTypedColumn[T, A] + ): TypedDataset[T] = orderByMany(ca) /** Orders the TypedDataset using the columns selected. */ def orderBy[A: CatalystOrdered, B: CatalystOrdered]( - ca: SortedTypedColumn[T, A], - cb: SortedTypedColumn[T, B] - ): TypedDataset[T] = orderByMany(ca, cb) - - /** Orders the TypedDataset using the columns selected. */ - def orderBy[A: CatalystOrdered, B: CatalystOrdered, C: CatalystOrdered]( - ca: SortedTypedColumn[T, A], - cb: SortedTypedColumn[T, B], - cc: SortedTypedColumn[T, C] - ): TypedDataset[T] = orderByMany(ca, cb, cc) - - /** Sort the dataset by any number of column expressions. - * Default sort order is ascending. - * {{{ - * d.orderByMany(d('a), d('b).desc, d('c).asc) - * }}} - */ + ca: SortedTypedColumn[T, A], + cb: SortedTypedColumn[T, B] + ): TypedDataset[T] = orderByMany(ca, cb) + + /** Orders the TypedDataset using the columns selected. */ + def orderBy[A: CatalystOrdered, B: CatalystOrdered, C: CatalystOrdered]( + ca: SortedTypedColumn[T, A], + cb: SortedTypedColumn[T, B], + cc: SortedTypedColumn[T, C] + ): TypedDataset[T] = orderByMany(ca, cb, cc) + + /** + * Sort the dataset by any number of column expressions. + * Default sort order is ascending. + * {{{ + * d.orderByMany(d('a), d('b).desc, d('c).asc) + * }}} + */ object orderByMany extends ProductArgs { - def applyProduct[U <: HList, O <: HList](columns: U) - (implicit + + def applyProduct[U <: HList, O <: HList]( + columns: U + )(implicit i0: Mapper.Aux[SortedTypedColumn.defaultAscendingPoly.type, U, O], i1: ToTraversable.Aux[O, List, SortedTypedColumn[T, _]] ): TypedDataset[T] = { - val sorted = dataset.toDF() - .orderBy(i0(columns).toList[SortedTypedColumn[T, _]].map(_.untyped):_*) + val sorted = dataset + .toDF() + .orderBy(i0(columns).toList[SortedTypedColumn[T, _]].map(_.untyped): _*) .as[T](TypedExpressionEncoder[T]) TypedDataset.create[T](sorted) } } - /** Returns a new Dataset as a tuple with the specified - * column dropped. - * Does not allow for dropping from a single column TypedDataset - * - * {{{ - * val d: TypedDataset[Foo(a: String, b: Int...)] = ??? - * val result = TypedDataset[(Int, ...)] = d.drop('a) - * }}} - * @param column column to drop specified as a Symbol - * @param i0 LabelledGeneric derived for T - * @param i1 Remover derived for TRep and column - * @param i2 values of T with column removed - * @param i3 tupler of values - * @param i4 evidence of encoder of the tupled values - * @tparam Out Tupled return type - * @tparam TRep shapeless' record representation of T - * @tparam Removed record of T with column removed - * @tparam ValuesFromRemoved values of T with column removed as an HList - * @tparam V value type of column in T - * @return - */ - def dropTupled[Out, TRep <: HList, Removed <: HList, ValuesFromRemoved <: HList, V] - (column: Witness.Lt[Symbol]) - (implicit + /** + * Returns a new Dataset as a tuple with the specified + * column dropped. + * Does not allow for dropping from a single column TypedDataset + * + * {{{ + * val d: TypedDataset[Foo(a: String, b: Int...)] = ??? + * val result = TypedDataset[(Int, ...)] = d.drop('a) + * }}} + * @param column column to drop specified as a Symbol + * @param i0 LabelledGeneric derived for T + * @param i1 Remover derived for TRep and column + * @param i2 values of T with column removed + * @param i3 tupler of values + * @param i4 evidence of encoder of the tupled values + * @tparam Out Tupled return type + * @tparam TRep shapeless' record representation of T + * @tparam Removed record of T with column removed + * @tparam ValuesFromRemoved values of T with column removed as an HList + * @tparam V value type of column in T + * @return + */ + def dropTupled[ + Out, + TRep <: HList, + Removed <: HList, + ValuesFromRemoved <: HList, + V + ](column: Witness.Lt[Symbol] + )(implicit i0: LabelledGeneric.Aux[T, TRep], i1: Remover.Aux[TRep, column.T, (V, Removed)], i2: Values.Aux[Removed, ValuesFromRemoved], i3: Tupler.Aux[ValuesFromRemoved, Out], i4: TypedEncoder[Out] ): TypedDataset[Out] = { - val dropped = dataset - .toDF() - .drop(column.value.name) - .as[Out](TypedExpressionEncoder[Out]) + val dropped = dataset + .toDF() + .drop(column.value.name) + .as[Out](TypedExpressionEncoder[Out]) - TypedDataset.create[Out](dropped) - } + TypedDataset.create[Out](dropped) + } /** - * Drops columns as necessary to return `U` - * - * @example - * {{{ - * case class X(i: Int, j: Int, k: Boolean) - * case class Y(i: Int, k: Boolean) - * val f: TypedDataset[X] = ??? - * val fNew: TypedDataset[Y] = f.drop[Y] - * }}} - * - * @tparam U the output type - * - * @see [[frameless.TypedDataset#project]] - */ - def drop[U](implicit projector: SmartProject[T,U]): TypedDataset[U] = project[U] - - /** Prepends a new column to the Dataset. - * - * {{{ - * case class X(i: Int, j: Int) - * val f: TypedDataset[X] = TypedDataset.create(X(1,1) :: X(1,1) :: X(1,10) :: Nil) - * val fNew: TypedDataset[(Int,Int,Boolean)] = f.withColumnTupled(f('j) === 10) - * }}} - */ - def withColumnTupled[A: TypedEncoder, H <: HList, FH <: HList, Out] - (ca: TypedColumn[T, A]) - (implicit + * Drops columns as necessary to return `U` + * + * @example + * {{{ + * case class X(i: Int, j: Int, k: Boolean) + * case class Y(i: Int, k: Boolean) + * val f: TypedDataset[X] = ??? + * val fNew: TypedDataset[Y] = f.drop[Y] + * }}} + * + * @tparam U the output type + * + * @see [[frameless.TypedDataset#project]] + */ + def drop[U]( + implicit + projector: SmartProject[T, U] + ): TypedDataset[U] = project[U] + + /** + * Prepends a new column to the Dataset. + * + * {{{ + * case class X(i: Int, j: Int) + * val f: TypedDataset[X] = TypedDataset.create(X(1,1) :: X(1,1) :: X(1,10) :: Nil) + * val fNew: TypedDataset[(Int,Int,Boolean)] = f.withColumnTupled(f('j) === 10) + * }}} + */ + def withColumnTupled[A: TypedEncoder, H <: HList, FH <: HList, Out]( + ca: TypedColumn[T, A] + )(implicit i0: Generic.Aux[T, H], i1: Prepend.Aux[H, A :: HNil, FH], i2: Tupler.Aux[FH, Out], i3: TypedEncoder[Out] ): TypedDataset[Out] = { - // Giving a random name to the new column (the proper name will be given by the Tuple-based encoder) - val selected = dataset.toDF().withColumn("I1X3T9CU1OP0128JYIO76TYZZA3AXHQ18RMI", ca.untyped) - .as[Out](TypedExpressionEncoder[Out]) + // Giving a random name to the new column (the proper name will be given by the Tuple-based encoder) + val selected = dataset + .toDF() + .withColumn("I1X3T9CU1OP0128JYIO76TYZZA3AXHQ18RMI", ca.untyped) + .as[Out](TypedExpressionEncoder[Out]) - TypedDataset.create[Out](selected) + TypedDataset.create[Out](selected) } - /** Returns a new [[frameless.TypedDataset]] with the specified column updated with a new value - * {{{ - * case class X(i: Int, j: Int) - * val f: TypedDataset[X] = TypedDataset.create(X(1,10) :: Nil) - * val fNew: TypedDataset[X] = f.withColumn('j, f('i)) // results in X(1, 1) :: Nil - * }}} - * @param column column given as a symbol to replace - * @param replacement column to replace the value with - * @param i0 Evidence that a column with the correct type and name exists - */ + /** + * Returns a new [[frameless.TypedDataset]] with the specified column updated with a new value + * {{{ + * case class X(i: Int, j: Int) + * val f: TypedDataset[X] = TypedDataset.create(X(1,10) :: Nil) + * val fNew: TypedDataset[X] = f.withColumn('j, f('i)) // results in X(1, 1) :: Nil + * }}} + * @param column column given as a symbol to replace + * @param replacement column to replace the value with + * @param i0 Evidence that a column with the correct type and name exists + */ def withColumnReplaced[A]( - column: Witness.Lt[Symbol], - replacement: TypedColumn[T, A] - )(implicit - i0: TypedColumn.Exists[T, column.T, A] - ): TypedDataset[T] = { - val updated = dataset.toDF().withColumn(column.value.name, replacement.untyped) + column: Witness.Lt[Symbol], + replacement: TypedColumn[T, A] + )(implicit + i0: TypedColumn.Exists[T, column.T, A] + ): TypedDataset[T] = { + val updated = dataset + .toDF() + .withColumn(column.value.name, replacement.untyped) .as[T](TypedExpressionEncoder[T]) TypedDataset.create[T](updated) } - /** Adds a column to a Dataset so long as the specified output type, `U`, has - * an extra column from `T` that has type `A`. - * - * @example - * {{{ - * case class X(i: Int, j: Int) - * case class Y(i: Int, j: Int, k: Boolean) - * val f: TypedDataset[X] = TypedDataset.create(X(1,1) :: X(1,1) :: X(1,10) :: Nil) - * val fNew: TypedDataset[Y] = f.withColumn[Y](f('j) === 10) - * }}} - * @param ca The typed column to add - * @param i0 TypeEncoder for output type U - * @param i1 TypeEncoder for added column type A - * @param i2 the LabelledGeneric derived for T - * @param i3 the LabelledGeneric derived for U - * @param i4 proof no fields have been removed - * @param i5 diff from T to U - * @param i6 keys from newFields - * @param i7 the one and only new key - * @param i8 the one and only new field enforcing the type of A exists - * @param i9 the keys of U - * @param iA allows for traversing the keys of U - * @tparam U the output type - * @tparam A The added column type - * @tparam TRep shapeless' record representation of T - * @tparam URep shapeless' record representation of U - * @tparam UKeys the keys of U as an HList - * @tparam NewFields the added fields to T to get U - * @tparam NewKeys the keys of NewFields as an HList - * @tparam NewKey the first, and only, key in NewKey - * - * @see [[frameless.TypedDataset.WithColumnApply#apply]] - */ + /** + * Adds a column to a Dataset so long as the specified output type, `U`, has + * an extra column from `T` that has type `A`. + * + * @example + * {{{ + * case class X(i: Int, j: Int) + * case class Y(i: Int, j: Int, k: Boolean) + * val f: TypedDataset[X] = TypedDataset.create(X(1,1) :: X(1,1) :: X(1,10) :: Nil) + * val fNew: TypedDataset[Y] = f.withColumn[Y](f('j) === 10) + * }}} + * @param ca The typed column to add + * @param i0 TypeEncoder for output type U + * @param i1 TypeEncoder for added column type A + * @param i2 the LabelledGeneric derived for T + * @param i3 the LabelledGeneric derived for U + * @param i4 proof no fields have been removed + * @param i5 diff from T to U + * @param i6 keys from newFields + * @param i7 the one and only new key + * @param i8 the one and only new field enforcing the type of A exists + * @param i9 the keys of U + * @param iA allows for traversing the keys of U + * @tparam U the output type + * @tparam A The added column type + * @tparam TRep shapeless' record representation of T + * @tparam URep shapeless' record representation of U + * @tparam UKeys the keys of U as an HList + * @tparam NewFields the added fields to T to get U + * @tparam NewKeys the keys of NewFields as an HList + * @tparam NewKey the first, and only, key in NewKey + * + * @see [[frameless.TypedDataset.WithColumnApply#apply]] + */ def withColumn[U] = new WithColumnApply[U] class WithColumnApply[U] { - def apply[A, TRep <: HList, URep <: HList, UKeys <: HList, NewFields <: HList, NewKeys <: HList, NewKey <: Symbol] - (ca: TypedColumn[T, A]) - (implicit - i0: TypedEncoder[U], - i1: TypedEncoder[A], - i2: LabelledGeneric.Aux[T, TRep], - i3: LabelledGeneric.Aux[U, URep], - i4: Diff.Aux[TRep, URep, HNil], - i5: Diff.Aux[URep, TRep, NewFields], - i6: Keys.Aux[NewFields, NewKeys], - i7: IsHCons.Aux[NewKeys, NewKey, HNil], - i8: IsHCons.Aux[NewFields, FieldType[NewKey, A], HNil], - i9: Keys.Aux[URep, UKeys], - iA: ToTraversable.Aux[UKeys, Seq, Symbol] - ): TypedDataset[U] = { + + def apply[ + A, + TRep <: HList, + URep <: HList, + UKeys <: HList, + NewFields <: HList, + NewKeys <: HList, + NewKey <: Symbol + ](ca: TypedColumn[T, A] + )(implicit + i0: TypedEncoder[U], + i1: TypedEncoder[A], + i2: LabelledGeneric.Aux[T, TRep], + i3: LabelledGeneric.Aux[U, URep], + i4: Diff.Aux[TRep, URep, HNil], + i5: Diff.Aux[URep, TRep, NewFields], + i6: Keys.Aux[NewFields, NewKeys], + i7: IsHCons.Aux[NewKeys, NewKey, HNil], + i8: IsHCons.Aux[NewFields, FieldType[NewKey, A], HNil], + i9: Keys.Aux[URep, UKeys], + iA: ToTraversable.Aux[UKeys, Seq, Symbol] + ): TypedDataset[U] = { val newColumnName = i7.head(i6()).name - val dfWithNewColumn = dataset - .toDF() - .withColumn(newColumnName, ca.untyped) + val dfWithNewColumn = dataset.toDF().withColumn(newColumnName, ca.untyped) val newColumns = i9.apply().to[Seq].map(_.name).map(dfWithNewColumn.col) - val selected = dfWithNewColumn - .select(newColumns: _*) - .as[U](TypedExpressionEncoder[U]) + val selected = + dfWithNewColumn.select(newColumns: _*).as[U](TypedExpressionEncoder[U]) TypedDataset.create[U](selected) } } /** - * Explodes a single column at a time. It only compiles if the type of column supports this operation. - * - * @example - * - * {{{ - * case class X(i: Int, j: Array[Int]) - * case class Y(i: Int, j: Int) - * - * val f: TypedDataset[X] = ??? - * val fNew: TypedDataset[Y] = f.explode('j).as[Y] - * }}} - * @param column the column we wish to explode - */ - def explode[A, TRep <: HList, V[_], OutMod <: HList, OutModValues <: HList, Out] - (column: Witness.Lt[Symbol]) - (implicit - i0: TypedColumn.Exists[T, column.T, V[A]], - i1: TypedEncoder[A], - i2: CatalystExplodableCollection[V], - i3: LabelledGeneric.Aux[T, TRep], - i4: Modifier.Aux[TRep, column.T, V[A], A, OutMod], - i5: Values.Aux[OutMod, OutModValues], - i6: Tupler.Aux[OutModValues, Out], - i7: TypedEncoder[Out] - ): TypedDataset[Out] = { - import org.apache.spark.sql.functions.{explode => sparkExplode} + * Explodes a single column at a time. It only compiles if the type of column supports this operation. + * + * @example + * + * {{{ + * case class X(i: Int, j: Array[Int]) + * case class Y(i: Int, j: Int) + * + * val f: TypedDataset[X] = ??? + * val fNew: TypedDataset[Y] = f.explode('j).as[Y] + * }}} + * @param column the column we wish to explode + */ + def explode[ + A, + TRep <: HList, + V[_], + OutMod <: HList, + OutModValues <: HList, + Out + ](column: Witness.Lt[Symbol] + )(implicit + i0: TypedColumn.Exists[T, column.T, V[A]], + i1: TypedEncoder[A], + i2: CatalystExplodableCollection[V], + i3: LabelledGeneric.Aux[T, TRep], + i4: Modifier.Aux[TRep, column.T, V[A], A, OutMod], + i5: Values.Aux[OutMod, OutModValues], + i6: Tupler.Aux[OutModValues, Out], + i7: TypedEncoder[Out] + ): TypedDataset[Out] = { + import org.apache.spark.sql.functions.{ explode => sparkExplode } val df = dataset.toDF() val trans = - df - .withColumn(column.value.name, sparkExplode(df(column.value.name))) + df.withColumn(column.value.name, sparkExplode(df(column.value.name))) .as[Out](TypedExpressionEncoder[Out]) TypedDataset.create[Out](trans) } /** - * Explodes a single column at a time. It only compiles if the type of column supports this operation. - * - * @example - * - * {{{ - * case class X(i: Int, j: Map[Int, Int]) - * case class Y(i: Int, j: (Int, Int)) - * - * val f: TypedDataset[X] = ??? - * val fNew: TypedDataset[Y] = f.explodeMap('j).as[Y] - * }}} - * @param column the column we wish to explode - */ - def explodeMap[A, B, V[_, _], TRep <: HList, OutMod <: HList, OutModValues <: HList, Out] - (column: Witness.Lt[Symbol]) - (implicit - i0: TypedColumn.Exists[T, column.T, V[A, B]], - i1: TypedEncoder[A], - i2: TypedEncoder[B], - i3: LabelledGeneric.Aux[T, TRep], - i4: Modifier.Aux[TRep, column.T, V[A,B], (A, B), OutMod], - i5: Values.Aux[OutMod, OutModValues], - i6: Tupler.Aux[OutModValues, Out], - i7: TypedEncoder[Out] - ): TypedDataset[Out] = { - import org.apache.spark.sql.functions.{explode => sparkExplode, struct => sparkStruct, col => sparkCol} + * Explodes a single column at a time. It only compiles if the type of column supports this operation. + * + * @example + * + * {{{ + * case class X(i: Int, j: Map[Int, Int]) + * case class Y(i: Int, j: (Int, Int)) + * + * val f: TypedDataset[X] = ??? + * val fNew: TypedDataset[Y] = f.explodeMap('j).as[Y] + * }}} + * @param column the column we wish to explode + */ + def explodeMap[ + A, + B, + V[_, _], + TRep <: HList, + OutMod <: HList, + OutModValues <: HList, + Out + ](column: Witness.Lt[Symbol] + )(implicit + i0: TypedColumn.Exists[T, column.T, V[A, B]], + i1: TypedEncoder[A], + i2: TypedEncoder[B], + i3: LabelledGeneric.Aux[T, TRep], + i4: Modifier.Aux[TRep, column.T, V[A, B], (A, B), OutMod], + i5: Values.Aux[OutMod, OutModValues], + i6: Tupler.Aux[OutModValues, Out], + i7: TypedEncoder[Out] + ): TypedDataset[Out] = { + import org.apache.spark.sql.functions.{ + explode => sparkExplode, + struct => sparkStruct, + col => sparkCol + } val df = dataset.toDF() // select all columns, all original columns and [key, value] columns appeared after the map explode @@ -1272,7 +1620,10 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val exploded // map explode explodes it into [key, value] columns // the only way to put it into a column is to create a struct - .withColumn(columnRenamed, sparkStruct(exploded("key"), exploded("value"))) + .withColumn( + columnRenamed, + sparkStruct(exploded("key"), exploded("value")) + ) // selecting only original columns, we don't need [key, value] columns left in the DataFrame after the map explode .select(columns: _*) // rename columns back and form the result @@ -1282,72 +1633,81 @@ class TypedDataset[T] protected[frameless](val dataset: Dataset[T])(implicit val } /** - * Flattens a column of type Option[A]. Compiles only if the selected column is of type Option[A]. - * - * - * @example - * - * {{{ - * case class X(i: Int, j: Option[Int]) - * case class Y(i: Int, j: Int) - * - * val f: TypedDataset[X] = ??? - * val fNew: TypedDataset[Y] = f.flattenOption('j).as[Y] - * }}} - * - * @param column the column we wish to flatten - */ - def flattenOption[A, TRep <: HList, V[_], OutMod <: HList, OutModValues <: HList, Out] - (column: Witness.Lt[Symbol]) - (implicit - i0: TypedColumn.Exists[T, column.T, V[A]], - i1: TypedEncoder[A], - i2: V[A] =:= Option[A], - i3: LabelledGeneric.Aux[T, TRep], - i4: Modifier.Aux[TRep, column.T, V[A], A, OutMod], - i5: Values.Aux[OutMod, OutModValues], - i6: Tupler.Aux[OutModValues, Out], - i7: TypedEncoder[Out] - ): TypedDataset[Out] = { + * Flattens a column of type Option[A]. Compiles only if the selected column is of type Option[A]. + * + * @example + * + * {{{ + * case class X(i: Int, j: Option[Int]) + * case class Y(i: Int, j: Int) + * + * val f: TypedDataset[X] = ??? + * val fNew: TypedDataset[Y] = f.flattenOption('j).as[Y] + * }}} + * + * @param column the column we wish to flatten + */ + def flattenOption[ + A, + TRep <: HList, + V[_], + OutMod <: HList, + OutModValues <: HList, + Out + ](column: Witness.Lt[Symbol] + )(implicit + i0: TypedColumn.Exists[T, column.T, V[A]], + i1: TypedEncoder[A], + i2: V[A] =:= Option[A], + i3: LabelledGeneric.Aux[T, TRep], + i4: Modifier.Aux[TRep, column.T, V[A], A, OutMod], + i5: Values.Aux[OutMod, OutModValues], + i6: Tupler.Aux[OutModValues, Out], + i7: TypedEncoder[Out] + ): TypedDataset[Out] = { val df = dataset.toDF() - val trans = df.filter(df(column.value.name).isNotNull). - as[Out](TypedExpressionEncoder[Out]) + val trans = df + .filter(df(column.value.name).isNotNull) + .as[Out](TypedExpressionEncoder[Out]) TypedDataset.create[Out](trans) } } object TypedDataset { - def create[A](data: Seq[A]) - (implicit + + def create[A]( + data: Seq[A] + )(implicit encoder: TypedEncoder[A], sqlContext: SparkSession ): TypedDataset[A] = { - val dataset = sqlContext.createDataset(data)(TypedExpressionEncoder[A]) + val dataset = sqlContext.createDataset(data)(TypedExpressionEncoder[A]) - TypedDataset.create[A](dataset) - } + TypedDataset.create[A](dataset) + } - def create[A](data: RDD[A]) - (implicit + def create[A]( + data: RDD[A] + )(implicit encoder: TypedEncoder[A], sqlContext: SparkSession ): TypedDataset[A] = { - val dataset = sqlContext.createDataset(data)(TypedExpressionEncoder[A]) + val dataset = sqlContext.createDataset(data)(TypedExpressionEncoder[A]) - TypedDataset.create[A](dataset) - } + TypedDataset.create[A](dataset) + } def create[A: TypedEncoder](dataset: Dataset[A]): TypedDataset[A] = createUnsafe(dataset.toDF()) /** - * Creates a [[frameless.TypedDataset]] from a Spark [[org.apache.spark.sql.DataFrame]]. - * Note that the names and types need to align! - * - * This is an unsafe operation: If the schemas do not align, - * the error will be captured at runtime (not during compilation). - */ + * Creates a [[frameless.TypedDataset]] from a Spark [[org.apache.spark.sql.DataFrame]]. + * Note that the names and types need to align! + * + * This is an unsafe operation: If the schemas do not align, + * the error will be captured at runtime (not during compilation). + */ def createUnsafe[A: TypedEncoder](df: DataFrame): TypedDataset[A] = { val e = TypedEncoder[A] val output: Seq[Attribute] = df.queryExecution.analyzed.output @@ -1359,7 +1719,8 @@ object TypedDataset { throw new IllegalStateException( s"Unsupported creation of TypedDataset with ${targetFields.size} column(s) " + s"from a DataFrame with ${output.size} columns. " + - "Try to `select()` the proper columns in the right order before calling `create()`.") + "Try to `select()` the proper columns in the right order before calling `create()`." + ) } // Adapt names if they are not the same (note: types still might not match) @@ -1369,7 +1730,7 @@ object TypedDataset { val canSelect = targetColNames.toSet.subsetOf(output.map(_.name).toSet) val reshaped = if (shouldReshape && canSelect) { - df.select(targetColNames.head, targetColNames.tail:_*) + df.select(targetColNames.head, targetColNames.tail: _*) } else if (shouldReshape) { df.toDF(targetColNames: _*) } else { @@ -1379,9 +1740,14 @@ object TypedDataset { new TypedDataset[A](reshaped.as[A](TypedExpressionEncoder[A])) } - /** Prefer `TypedDataset.create` over `TypedDataset.unsafeCreate` unless you - * know what you are doing. */ - @deprecated("Prefer TypedDataset.create over TypedDataset.unsafeCreate", "0.3.0") + /** + * Prefer `TypedDataset.create` over `TypedDataset.unsafeCreate` unless you + * know what you are doing. + */ + @deprecated( + "Prefer TypedDataset.create over TypedDataset.unsafeCreate", + "0.3.0" + ) def unsafeCreate[A: TypedEncoder](dataset: Dataset[A]): TypedDataset[A] = { new TypedDataset[A](dataset) } diff --git a/dataset/src/main/scala/frameless/TypedDatasetForwarded.scala b/dataset/src/main/scala/frameless/TypedDatasetForwarded.scala index 119256279..0856732f2 100644 --- a/dataset/src/main/scala/frameless/TypedDatasetForwarded.scala +++ b/dataset/src/main/scala/frameless/TypedDatasetForwarded.scala @@ -6,366 +6,429 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.streaming.DataStreamWriter import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{DataFrame, DataFrameWriter, FramelessInternals, SQLContext, SparkSession} +import org.apache.spark.sql.{ + DataFrame, + DataFrameWriter, + FramelessInternals, + SQLContext, + SparkSession +} import org.apache.spark.storage.StorageLevel import scala.util.Random -/** This trait implements [[TypedDataset]] methods that have the same signature - * than their `Dataset` equivalent. Each method simply forwards the call to the - * underlying `Dataset`. - * - * Documentation marked "apache/spark" is thanks to apache/spark Contributors - * at https://github.com/apache/spark, licensed under Apache v2.0 available at - * http://www.apache.org/licenses/LICENSE-2.0 - */ +/** + * This trait implements [[TypedDataset]] methods that have the same signature + * than their `Dataset` equivalent. Each method simply forwards the call to the + * underlying `Dataset`. + * + * Documentation marked "apache/spark" is thanks to apache/spark Contributors + * at https://github.com/apache/spark, licensed under Apache v2.0 available at + * http://www.apache.org/licenses/LICENSE-2.0 + */ trait TypedDatasetForwarded[T] { self: TypedDataset[T] => override def toString: String = dataset.toString /** - * Returns a `SparkSession` from this [[TypedDataset]]. - */ + * Returns a `SparkSession` from this [[TypedDataset]]. + */ def sparkSession: SparkSession = dataset.sparkSession /** - * Returns a `SQLContext` from this [[TypedDataset]]. - */ + * Returns a `SQLContext` from this [[TypedDataset]]. + */ def sqlContext: SQLContext = FramelessInternals.sqlContext(dataset) /** - * Returns the schema of this Dataset. - * - * apache/spark - */ + * Returns the schema of this Dataset. + * + * apache/spark + */ def schema: StructType = dataset.schema - /** Prints the schema of the underlying `Dataset` to the console in a nice tree format. - * - * apache/spark + /** + * Prints the schema of the underlying `Dataset` to the console in a nice tree format. + * + * apache/spark */ def printSchema(): Unit = dataset.printSchema() - /** Prints the plans (logical and physical) to the console for debugging purposes. - * - * apache/spark + /** + * Prints the plans (logical and physical) to the console for debugging purposes. + * + * apache/spark */ def explain(extended: Boolean = false): Unit = dataset.explain(extended) /** - * Returns a `QueryExecution` from this [[TypedDataset]]. - * - * It is the primary workflow for executing relational queries using Spark. Designed to allow easy - * access to the intermediate phases of query execution for developers. - * - * apache/spark - */ + * Returns a `QueryExecution` from this [[TypedDataset]]. + * + * It is the primary workflow for executing relational queries using Spark. Designed to allow easy + * access to the intermediate phases of query execution for developers. + * + * apache/spark + */ def queryExecution: QueryExecution = dataset.queryExecution - /** Converts this strongly typed collection of data to generic Dataframe. In contrast to the - * strongly typed objects that Dataset operations work on, a Dataframe returns generic Row - * objects that allow fields to be accessed by ordinal or name. - * - * apache/spark - */ + /** + * Converts this strongly typed collection of data to generic Dataframe. In contrast to the + * strongly typed objects that Dataset operations work on, a Dataframe returns generic Row + * objects that allow fields to be accessed by ordinal or name. + * + * apache/spark + */ def toDF(): DataFrame = dataset.toDF() - /** Converts this [[TypedDataset]] to an RDD. - * - * apache/spark - */ + /** + * Converts this [[TypedDataset]] to an RDD. + * + * apache/spark + */ def rdd: RDD[T] = dataset.rdd - /** Returns a new [[TypedDataset]] that has exactly `numPartitions` partitions. - * - * apache/spark - */ + /** + * Returns a new [[TypedDataset]] that has exactly `numPartitions` partitions. + * + * apache/spark + */ def repartition(numPartitions: Int): TypedDataset[T] = TypedDataset.create(dataset.repartition(numPartitions)) - /** - * Get the [[TypedDataset]]'s current storage level, or StorageLevel.NONE if not persisted. - * - * apache/spark - */ + * Get the [[TypedDataset]]'s current storage level, or StorageLevel.NONE if not persisted. + * + * apache/spark + */ def storageLevel(): StorageLevel = dataset.storageLevel /** - * Returns the content of the [[TypedDataset]] as a Dataset of JSON strings. - * - * apache/spark - */ + * Returns the content of the [[TypedDataset]] as a Dataset of JSON strings. + * + * apache/spark + */ def toJSON: TypedDataset[String] = TypedDataset.create(dataset.toJSON) /** - * Interface for saving the content of the non-streaming [[TypedDataset]] out into external storage. - * - * apache/spark - */ + * Interface for saving the content of the non-streaming [[TypedDataset]] out into external storage. + * + * apache/spark + */ def write: DataFrameWriter[T] = dataset.write /** - * Interface for saving the content of the streaming Dataset out into external storage. - * - * apache/spark - */ + * Interface for saving the content of the streaming Dataset out into external storage. + * + * apache/spark + */ def writeStream: DataStreamWriter[T] = dataset.writeStream - - /** Returns a new [[TypedDataset]] that has exactly `numPartitions` partitions. - * Similar to coalesce defined on an RDD, this operation results in a narrow dependency, e.g. - * if you go from 1000 partitions to 100 partitions, there will not be a shuffle, instead each of - * the 100 new partitions will claim 10 of the current partitions. - * - * apache/spark - */ + + /** + * Returns a new [[TypedDataset]] that has exactly `numPartitions` partitions. + * Similar to coalesce defined on an RDD, this operation results in a narrow dependency, e.g. + * if you go from 1000 partitions to 100 partitions, there will not be a shuffle, instead each of + * the 100 new partitions will claim 10 of the current partitions. + * + * apache/spark + */ def coalesce(numPartitions: Int): TypedDataset[T] = TypedDataset.create(dataset.coalesce(numPartitions)) /** - * Returns an `Array` that contains all column names in this [[TypedDataset]]. - */ + * Returns an `Array` that contains all column names in this [[TypedDataset]]. + */ def columns: Array[String] = dataset.columns - /** Concise syntax for chaining custom transformations. - * - * apache/spark - */ + /** + * Concise syntax for chaining custom transformations. + * + * apache/spark + */ def transform[U](t: TypedDataset[T] => TypedDataset[U]): TypedDataset[U] = t(this) - /** Returns a new Dataset by taking the first `n` rows. The difference between this function - * and `head` is that `head` is an action and returns an array (by triggering query execution) - * while `limit` returns a new Dataset. - * - * apache/spark - */ + /** + * Returns a new Dataset by taking the first `n` rows. The difference between this function + * and `head` is that `head` is an action and returns an array (by triggering query execution) + * while `limit` returns a new Dataset. + * + * apache/spark + */ def limit(n: Int): TypedDataset[T] = TypedDataset.create(dataset.limit(n)) - /** Returns a new [[TypedDataset]] by sampling a fraction of records. - * - * apache/spark - */ - def sample(withReplacement: Boolean, fraction: Double, seed: Long = Random.nextLong()): TypedDataset[T] = + /** + * Returns a new [[TypedDataset]] by sampling a fraction of records. + * + * apache/spark + */ + def sample( + withReplacement: Boolean, + fraction: Double, + seed: Long = Random.nextLong() + ): TypedDataset[T] = TypedDataset.create(dataset.sample(withReplacement, fraction, seed)) - /** Returns a new [[TypedDataset]] that contains only the unique elements of this [[TypedDataset]]. - * - * Note that, equality checking is performed directly on the encoded representation of the data - * and thus is not affected by a custom `equals` function defined on `T`. - * - * apache/spark - */ + /** + * Returns a new [[TypedDataset]] that contains only the unique elements of this [[TypedDataset]]. + * + * Note that, equality checking is performed directly on the encoded representation of the data + * and thus is not affected by a custom `equals` function defined on `T`. + * + * apache/spark + */ def distinct: TypedDataset[T] = TypedDataset.create(dataset.distinct()) /** - * Returns a best-effort snapshot of the files that compose this [[TypedDataset]]. This method simply - * asks each constituent BaseRelation for its respective files and takes the union of all results. - * Depending on the source relations, this may not find all input files. Duplicates are removed. - * - * apache/spark - */ + * Returns a best-effort snapshot of the files that compose this [[TypedDataset]]. This method simply + * asks each constituent BaseRelation for its respective files and takes the union of all results. + * Depending on the source relations, this may not find all input files. Duplicates are removed. + * + * apache/spark + */ def inputFiles: Array[String] = dataset.inputFiles /** - * Returns true if the `collect` and `take` methods can be run locally - * (without any Spark executors). - * - * apache/spark - */ + * Returns true if the `collect` and `take` methods can be run locally + * (without any Spark executors). + * + * apache/spark + */ def isLocal: Boolean = dataset.isLocal /** - * Returns true if this [[TypedDataset]] contains one or more sources that continuously - * return data as it arrives. A [[TypedDataset]] that reads data from a streaming source - * must be executed as a `StreamingQuery` using the `start()` method in - * `DataStreamWriter`. Methods that return a single answer, e.g. `count()` or - * `collect()`, will throw an `AnalysisException` when there is a streaming - * source present. - * - * apache/spark - */ + * Returns true if this [[TypedDataset]] contains one or more sources that continuously + * return data as it arrives. A [[TypedDataset]] that reads data from a streaming source + * must be executed as a `StreamingQuery` using the `start()` method in + * `DataStreamWriter`. Methods that return a single answer, e.g. `count()` or + * `collect()`, will throw an `AnalysisException` when there is a streaming + * source present. + * + * apache/spark + */ def isStreaming: Boolean = dataset.isStreaming - /** Returns a new [[TypedDataset]] that contains only the elements of this [[TypedDataset]] that are also - * present in `other`. - * - * Note that, equality checking is performed directly on the encoded representation of the data - * and thus is not affected by a custom `equals` function defined on `T`. - * - * apache/spark - */ + /** + * Returns a new [[TypedDataset]] that contains only the elements of this [[TypedDataset]] that are also + * present in `other`. + * + * Note that, equality checking is performed directly on the encoded representation of the data + * and thus is not affected by a custom `equals` function defined on `T`. + * + * apache/spark + */ def intersect(other: TypedDataset[T]): TypedDataset[T] = TypedDataset.create(dataset.intersect(other.dataset)) /** - * Randomly splits this [[TypedDataset]] with the provided weights. - * Weights for splits, will be normalized if they don't sum to 1. - * - * apache/spark - */ + * Randomly splits this [[TypedDataset]] with the provided weights. + * Weights for splits, will be normalized if they don't sum to 1. + * + * apache/spark + */ // $COVERAGE-OFF$ We can not test this method because it is non-deterministic. def randomSplit(weights: Array[Double]): Array[TypedDataset[T]] = dataset.randomSplit(weights).map(TypedDataset.create[T]) // $COVERAGE-ON$ /** - * Randomly splits this [[TypedDataset]] with the provided weights. - * Weights for splits, will be normalized if they don't sum to 1. - * - * apache/spark - */ + * Randomly splits this [[TypedDataset]] with the provided weights. + * Weights for splits, will be normalized if they don't sum to 1. + * + * apache/spark + */ def randomSplit(weights: Array[Double], seed: Long): Array[TypedDataset[T]] = dataset.randomSplit(weights, seed).map(TypedDataset.create[T]) /** - * Returns a Java list that contains randomly split [[TypedDataset]] with the provided weights. - * Weights for splits, will be normalized if they don't sum to 1. - * - * apache/spark - */ - def randomSplitAsList(weights: Array[Double], seed: Long): util.List[TypedDataset[T]] = { + * Returns a Java list that contains randomly split [[TypedDataset]] with the provided weights. + * Weights for splits, will be normalized if they don't sum to 1. + * + * apache/spark + */ + def randomSplitAsList( + weights: Array[Double], + seed: Long + ): util.List[TypedDataset[T]] = { val values = randomSplit(weights, seed) java.util.Arrays.asList(values: _*) } - - /** Returns a new Dataset containing rows in this Dataset but not in another Dataset. - * This is equivalent to `EXCEPT` in SQL. - * - * Note that, equality checking is performed directly on the encoded representation of the data - * and thus is not affected by a custom `equals` function defined on `T`. - * - * apache/spark - */ + /** + * Returns a new Dataset containing rows in this Dataset but not in another Dataset. + * This is equivalent to `EXCEPT` in SQL. + * + * Note that, equality checking is performed directly on the encoded representation of the data + * and thus is not affected by a custom `equals` function defined on `T`. + * + * apache/spark + */ def except(other: TypedDataset[T]): TypedDataset[T] = TypedDataset.create(dataset.except(other.dataset)) - /** Persist this [[TypedDataset]] with the default storage level (`MEMORY_AND_DISK`). - * - * apache/spark - */ + /** + * Persist this [[TypedDataset]] with the default storage level (`MEMORY_AND_DISK`). + * + * apache/spark + */ def cache(): TypedDataset[T] = TypedDataset.create(dataset.cache()) - /** Persist this [[TypedDataset]] with the given storage level. - * @param newLevel One of: `MEMORY_ONLY`, `MEMORY_AND_DISK`, `MEMORY_ONLY_SER`, - * `MEMORY_AND_DISK_SER`, `DISK_ONLY`, `MEMORY_ONLY_2`, `MEMORY_AND_DISK_2`, etc. - * - * apache/spark - */ - def persist(newLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK): TypedDataset[T] = + /** + * Persist this [[TypedDataset]] with the given storage level. + * @param newLevel One of: `MEMORY_ONLY`, `MEMORY_AND_DISK`, `MEMORY_ONLY_SER`, + * `MEMORY_AND_DISK_SER`, `DISK_ONLY`, `MEMORY_ONLY_2`, `MEMORY_AND_DISK_2`, etc. + * + * apache/spark + */ + def persist( + newLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK + ): TypedDataset[T] = TypedDataset.create(dataset.persist(newLevel)) - /** Mark the [[TypedDataset]] as non-persistent, and remove all blocks for it from memory and disk. - * @param blocking Whether to block until all blocks are deleted. - * - * apache/spark - */ + /** + * Mark the [[TypedDataset]] as non-persistent, and remove all blocks for it from memory and disk. + * @param blocking Whether to block until all blocks are deleted. + * + * apache/spark + */ def unpersist(blocking: Boolean = false): TypedDataset[T] = TypedDataset.create(dataset.unpersist(blocking)) // $COVERAGE-OFF$ We do not test deprecated method since forwarded methods are tested. - @deprecated("deserialized methods have moved to a separate section to highlight their runtime overhead", "0.4.0") + @deprecated( + "deserialized methods have moved to a separate section to highlight their runtime overhead", + "0.4.0" + ) def map[U: TypedEncoder](func: T => U): TypedDataset[U] = deserialized.map(func) - @deprecated("deserialized methods have moved to a separate section to highlight their runtime overhead", "0.4.0") - def mapPartitions[U: TypedEncoder](func: Iterator[T] => Iterator[U]): TypedDataset[U] = + @deprecated( + "deserialized methods have moved to a separate section to highlight their runtime overhead", + "0.4.0" + ) + def mapPartitions[U: TypedEncoder]( + func: Iterator[T] => Iterator[U] + ): TypedDataset[U] = deserialized.mapPartitions(func) - @deprecated("deserialized methods have moved to a separate section to highlight their runtime overhead", "0.4.0") + @deprecated( + "deserialized methods have moved to a separate section to highlight their runtime overhead", + "0.4.0" + ) def flatMap[U: TypedEncoder](func: T => TraversableOnce[U]): TypedDataset[U] = deserialized.flatMap(func) - @deprecated("deserialized methods have moved to a separate section to highlight their runtime overhead", "0.4.0") + @deprecated( + "deserialized methods have moved to a separate section to highlight their runtime overhead", + "0.4.0" + ) def filter(func: T => Boolean): TypedDataset[T] = deserialized.filter(func) - @deprecated("deserialized methods have moved to a separate section to highlight their runtime overhead", "0.4.0") + @deprecated( + "deserialized methods have moved to a separate section to highlight their runtime overhead", + "0.4.0" + ) def reduceOption[F[_]: SparkDelay](func: (T, T) => T): F[Option[T]] = deserialized.reduceOption(func) // $COVERAGE-ON$ - /** Methods on `TypedDataset[T]` that go through a full serialization and - * deserialization of `T`, and execute outside of the Catalyst runtime. - * - * @example The correct way to do a projection on a single column is to - * use the `select` method as follows: - * - * {{{ - * ds: TypedDataset[(String, String, String)] -> ds.select(ds('_2)).run() - * }}} - * - * Spark provides an alternative way to obtain the same resulting `Dataset`, - * using the `map` method: - * - * {{{ - * ds: TypedDataset[(String, String, String)] -> ds.deserialized.map(_._2).run() - * }}} - * - * This second approach is however substantially slower than the first one, - * and should be avoided as possible. Indeed, under the hood this `map` will - * deserialize the entire `Tuple3` to an full JVM object, call the apply - * method of the `_._2` closure on it, and serialize the resulting String back - * to its Catalyst representation. - */ + /** + * Methods on `TypedDataset[T]` that go through a full serialization and + * deserialization of `T`, and execute outside of the Catalyst runtime. + * + * @example The correct way to do a projection on a single column is to + * use the `select` method as follows: + * + * {{{ + * ds: TypedDataset[(String, String, String)] -> ds.select(ds('_2)).run() + * }}} + * + * Spark provides an alternative way to obtain the same resulting `Dataset`, + * using the `map` method: + * + * {{{ + * ds: TypedDataset[(String, String, String)] -> ds.deserialized.map(_._2).run() + * }}} + * + * This second approach is however substantially slower than the first one, + * and should be avoided as possible. Indeed, under the hood this `map` will + * deserialize the entire `Tuple3` to an full JVM object, call the apply + * method of the `_._2` closure on it, and serialize the resulting String back + * to its Catalyst representation. + */ object deserialized { - /** Returns a new [[TypedDataset]] that contains the result of applying `func` to each element. - * - * apache/spark - */ + + /** + * Returns a new [[TypedDataset]] that contains the result of applying `func` to each element. + * + * apache/spark + */ def map[U: TypedEncoder](func: T => U): TypedDataset[U] = TypedDataset.create(self.dataset.map(func)(TypedExpressionEncoder[U])) - /** Returns a new [[TypedDataset]] that contains the result of applying `func` to each partition. - * - * apache/spark - */ - def mapPartitions[U: TypedEncoder](func: Iterator[T] => Iterator[U]): TypedDataset[U] = - TypedDataset.create(self.dataset.mapPartitions(func)(TypedExpressionEncoder[U])) - - /** Returns a new [[TypedDataset]] by first applying a function to all elements of this [[TypedDataset]], - * and then flattening the results. - * - * apache/spark - */ - def flatMap[U: TypedEncoder](func: T => TraversableOnce[U]): TypedDataset[U] = + /** + * Returns a new [[TypedDataset]] that contains the result of applying `func` to each partition. + * + * apache/spark + */ + def mapPartitions[U: TypedEncoder]( + func: Iterator[T] => Iterator[U] + ): TypedDataset[U] = + TypedDataset.create( + self.dataset.mapPartitions(func)(TypedExpressionEncoder[U]) + ) + + /** + * Returns a new [[TypedDataset]] by first applying a function to all elements of this [[TypedDataset]], + * and then flattening the results. + * + * apache/spark + */ + def flatMap[U: TypedEncoder]( + func: T => TraversableOnce[U] + ): TypedDataset[U] = TypedDataset.create(self.dataset.flatMap(func)(TypedExpressionEncoder[U])) - /** Returns a new [[TypedDataset]] that only contains elements where `func` returns `true`. - * - * apache/spark - */ + /** + * Returns a new [[TypedDataset]] that only contains elements where `func` returns `true`. + * + * apache/spark + */ def filter(func: T => Boolean): TypedDataset[T] = TypedDataset.create(self.dataset.filter(func)) - /** Optionally reduces the elements of this [[TypedDataset]] using the specified binary function. The given - * `func` must be commutative and associative or the result may be non-deterministic. - * - * Differs from `Dataset#reduce` by wrapping its result into an `Option` and an effect-suspending `F`. - */ - def reduceOption[F[_]](func: (T, T) => T)(implicit F: SparkDelay[F]): F[Option[T]] = + /** + * Optionally reduces the elements of this [[TypedDataset]] using the specified binary function. The given + * `func` must be commutative and associative or the result may be non-deterministic. + * + * Differs from `Dataset#reduce` by wrapping its result into an `Option` and an effect-suspending `F`. + */ + def reduceOption[F[_]]( + func: (T, T) => T + )(implicit + F: SparkDelay[F] + ): F[Option[T]] = F.delay { try { Option(self.dataset.reduce(func)) diff --git a/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala b/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala index 1c2e3bc38..ba51a6ef3 100644 --- a/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala +++ b/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala @@ -1,19 +1,24 @@ package frameless -import org.apache.spark.sql.{Encoder, FramelessInternals} +import org.apache.spark.sql.{ Encoder, FramelessInternals } import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal -import org.apache.spark.sql.catalyst.expressions.{BoundReference, CreateNamedStruct, If} +import org.apache.spark.sql.catalyst.expressions.{ + BoundReference, + CreateNamedStruct, + If +} import org.apache.spark.sql.types.StructType object TypedExpressionEncoder { - /** In Spark, DataFrame has always schema of StructType - * - * DataFrames of primitive types become records - * with a single field called "value" set in ExpressionEncoder. - */ + /** + * In Spark, DataFrame has always schema of StructType + * + * DataFrames of primitive types become records + * with a single field called "value" set in ExpressionEncoder. + */ def targetStructType[A](encoder: TypedEncoder[A]): StructType = - encoder.catalystRepr match { + encoder.catalystRepr match { case x: StructType => if (encoder.nullable) StructType(x.fields.map(_.copy(nullable = true))) else x @@ -21,7 +26,10 @@ object TypedExpressionEncoder { case dt => new StructType().add("value", dt, nullable = encoder.nullable) } - def apply[T](implicit encoder: TypedEncoder[T]): Encoder[T] = { + def apply[T]( + implicit + encoder: TypedEncoder[T] + ): Encoder[T] = { val in = BoundReference(0, encoder.jvmRepr, encoder.nullable) val (out, serializer) = encoder.toCatalyst(in) match { @@ -45,4 +53,3 @@ object TypedExpressionEncoder { ) } } - diff --git a/dataset/src/main/scala/frameless/ops/GroupByOps.scala b/dataset/src/main/scala/frameless/ops/GroupByOps.scala index ccbacce8c..e6f51a407 100644 --- a/dataset/src/main/scala/frameless/ops/GroupByOps.scala +++ b/dataset/src/main/scala/frameless/ops/GroupByOps.scala @@ -3,36 +3,54 @@ package ops import org.apache.spark.sql.catalyst.analysis.UnresolvedAlias import org.apache.spark.sql.catalyst.plans.logical.Project -import org.apache.spark.sql.{Column, Dataset, FramelessInternals, RelationalGroupedDataset} +import org.apache.spark.sql.{ + Column, + Dataset, + FramelessInternals, + RelationalGroupedDataset +} import shapeless._ -import shapeless.ops.hlist.{Length, Mapped, Prepend, ToList, ToTraversable, Tupler} +import shapeless.ops.hlist.{ + Length, + Mapped, + Prepend, + ToList, + ToTraversable, + Tupler +} -class GroupedByManyOps[T, TK <: HList, K <: HList, KT] - (self: TypedDataset[T], groupedBy: TK) - (implicit +class GroupedByManyOps[T, TK <: HList, K <: HList, KT]( + self: TypedDataset[T], + groupedBy: TK + )(implicit i0: ColumnTypes.Aux[T, TK, K], i1: ToTraversable.Aux[TK, List, UntypedExpression[T]], - i3: Tupler.Aux[K, KT] - ) extends AggregatingOps[T, TK, K, KT](self, groupedBy, (dataset, cols) => dataset.groupBy(cols: _*)) { + i3: Tupler.Aux[K, KT]) + extends AggregatingOps[T, TK, K, KT]( + self, + groupedBy, + (dataset, cols) => dataset.groupBy(cols: _*) + ) { + object agg extends ProductArgs { - def applyProduct[TC <: HList, C <: HList, Out0 <: HList, Out1] - (columns: TC) - (implicit + + def applyProduct[TC <: HList, C <: HList, Out0 <: HList, Out1]( + columns: TC + )(implicit i3: AggregateTypes.Aux[T, TC, C], i4: Prepend.Aux[K, C, Out0], i5: Tupler.Aux[Out0, Out1], i6: TypedEncoder[Out1], i7: ToTraversable.Aux[TC, List, UntypedExpression[T]] ): TypedDataset[Out1] = { - aggregate[TC, Out1](columns) - } + aggregate[TC, Out1](columns) + } } } class GroupedBy1Ops[K1, V]( - self: TypedDataset[V], - g1: TypedColumn[V, K1] -) { + self: TypedDataset[V], + g1: TypedColumn[V, K1]) { private def underlying = new GroupedByManyOps(self, g1 :: HNil) private implicit def eg1 = g1.uencoder @@ -41,49 +59,77 @@ class GroupedBy1Ops[K1, V]( 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[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[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 + 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[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 + 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[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 + 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) } - /** Methods on `TypedDataset[T]` that go through a full serialization and - * deserialization of `T`, and execute outside of the Catalyst runtime. - */ + /** + * Methods on `TypedDataset[T]` that go through a full serialization and + * deserialization of `T`, and execute outside of the Catalyst runtime. + */ object deserialized { - def mapGroups[U: TypedEncoder](f: (K1, Iterator[V]) => U): TypedDataset[U] = { + + def mapGroups[U: TypedEncoder]( + f: (K1, Iterator[V]) => U + ): TypedDataset[U] = { underlying.deserialized.mapGroups(AggregatingOps.tuple1(f)) } - def flatMapGroups[U: TypedEncoder](f: (K1, Iterator[V]) => TraversableOnce[U]): TypedDataset[U] = { + def flatMapGroups[U: TypedEncoder]( + f: (K1, Iterator[V]) => TraversableOnce[U] + ): TypedDataset[U] = { underlying.deserialized.flatMapGroups(AggregatingOps.tuple1(f)) } } - def pivot[P: CatalystPivotable](pivotColumn: TypedColumn[V, P]): PivotNotValues[V, TypedColumn[V,K1] :: HNil, P] = + def pivot[P: CatalystPivotable]( + pivotColumn: TypedColumn[V, P] + ): PivotNotValues[V, TypedColumn[V, K1] :: HNil, P] = PivotNotValues(self, g1 :: HNil, pivotColumn) } - class GroupedBy2Ops[K1, K2, V]( - self: TypedDataset[V], - g1: TypedColumn[V, K1], - g2: TypedColumn[V, K2] -) { + self: TypedDataset[V], + 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 @@ -93,58 +139,90 @@ class GroupedBy2Ops[K1, K2, V]( 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[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[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 + 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[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]( + 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[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 + 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) } - - /** Methods on `TypedDataset[T]` that go through a full serialization and - * deserialization of `T`, and execute outside of the Catalyst runtime. - */ + /** + * Methods on `TypedDataset[T]` that go through a full serialization and + * deserialization of `T`, and execute outside of the Catalyst runtime. + */ object deserialized { - def mapGroups[U: TypedEncoder](f: ((K1, K2), Iterator[V]) => U): TypedDataset[U] = { + + def mapGroups[U: TypedEncoder]( + f: ((K1, K2), Iterator[V]) => U + ): TypedDataset[U] = { underlying.deserialized.mapGroups(f) } - def flatMapGroups[U: TypedEncoder](f: ((K1, K2), Iterator[V]) => TraversableOnce[U]): TypedDataset[U] = { + def flatMapGroups[U: TypedEncoder]( + f: ((K1, K2), Iterator[V]) => TraversableOnce[U] + ): TypedDataset[U] = { underlying.deserialized.flatMapGroups(f) } } - def pivot[P: CatalystPivotable](pivotColumn: TypedColumn[V, P]): - PivotNotValues[V, TypedColumn[V,K1] :: TypedColumn[V, K2] :: HNil, P] = - PivotNotValues(self, g1 :: g2 :: HNil, pivotColumn) + def pivot[P: CatalystPivotable]( + pivotColumn: TypedColumn[V, P] + ): PivotNotValues[V, TypedColumn[V, K1] :: TypedColumn[V, K2] :: HNil, P] = + PivotNotValues(self, g1 :: g2 :: HNil, pivotColumn) } -private[ops] abstract class AggregatingOps[T, TK <: HList, K <: HList, KT] - (self: TypedDataset[T], groupedBy: TK, groupingFunc: (Dataset[T], Seq[Column]) => RelationalGroupedDataset) - (implicit +private[ops] abstract class AggregatingOps[T, TK <: HList, K <: HList, KT]( + self: TypedDataset[T], + groupedBy: TK, + groupingFunc: (Dataset[T], Seq[Column]) => RelationalGroupedDataset + )(implicit i0: ColumnTypes.Aux[T, TK, K], i1: ToTraversable.Aux[TK, List, UntypedExpression[T]], - i2: Tupler.Aux[K, KT] - ) { - def aggregate[TC <: HList, Out1](columns: TC) - (implicit - i7: TypedEncoder[Out1], - i8: ToTraversable.Aux[TC, List, UntypedExpression[T]] - ): TypedDataset[Out1] = { - def expr(c: UntypedExpression[T]): Column = FramelessInternals.column(c.expr) + i2: Tupler.Aux[K, KT]) { + + def aggregate[TC <: HList, Out1]( + columns: TC + )(implicit + i7: TypedEncoder[Out1], + i8: ToTraversable.Aux[TC, List, UntypedExpression[T]] + ): TypedDataset[Out1] = { + def expr(c: UntypedExpression[T]): Column = + FramelessInternals.column(c.expr) val groupByExprs = groupedBy.toList[UntypedExpression[T]].map(expr) val aggregates = @@ -159,25 +237,32 @@ private[ops] abstract class AggregatingOps[T, TK <: HList, K <: HList, KT] TypedDataset.create[Out1](aggregated) } - /** Methods on `TypedDataset[T]` that go through a full serialization and - * deserialization of `T`, and execute outside of the Catalyst runtime. - */ + /** + * Methods on `TypedDataset[T]` that go through a full serialization and + * deserialization of `T`, and execute outside of the Catalyst runtime. + */ object deserialized { + def mapGroups[U: TypedEncoder]( - f: (KT, Iterator[T]) => U - )(implicit e: TypedEncoder[KT]): TypedDataset[U] = { + f: (KT, Iterator[T]) => U + )(implicit + e: TypedEncoder[KT] + ): TypedDataset[U] = { val func = (key: KT, it: Iterator[T]) => Iterator(f(key, it)) flatMapGroups(func) } def flatMapGroups[U: TypedEncoder]( - f: (KT, Iterator[T]) => TraversableOnce[U] - )(implicit e: TypedEncoder[KT]): TypedDataset[U] = { + f: (KT, Iterator[T]) => TraversableOnce[U] + )(implicit + e: TypedEncoder[KT] + ): TypedDataset[U] = { implicit val tendcoder = self.encoder val cols = groupedBy.toList[UntypedExpression[T]] val logicalPlan = FramelessInternals.logicalPlan(self.dataset) - val withKeyColumns = logicalPlan.output ++ cols.map(_.expr).map(UnresolvedAlias(_)) + val withKeyColumns = + logicalPlan.output ++ cols.map(_.expr).map(UnresolvedAlias(_)) val withKey = Project(withKeyColumns, logicalPlan) val executed = FramelessInternals.executePlan(self.dataset, withKey) val keyAttributes = executed.analyzed.output.takeRight(cols.size) @@ -188,7 +273,11 @@ private[ops] abstract class AggregatingOps[T, TK <: HList, K <: HList, KT] keyAttributes, dataAttributes, executed.analyzed - )(TypedExpressionEncoder[KT], TypedExpressionEncoder[T], TypedExpressionEncoder[U]) + )( + TypedExpressionEncoder[KT], + TypedExpressionEncoder[T], + TypedExpressionEncoder[U] + ) val groupedAndFlatMapped = FramelessInternals.mkDataset( self.dataset, @@ -201,66 +290,97 @@ private[ops] abstract class AggregatingOps[T, TK <: HList, K <: HList, KT] } private def retainGroupColumns: Boolean = { - FramelessInternals.getConf(self.dataset, "spark.sql.retainGroupColumns", "true").toBoolean + FramelessInternals + .getConf(self.dataset, "spark.sql.retainGroupColumns", "true") + .toBoolean } - def pivot[P: CatalystPivotable](pivotColumn: TypedColumn[T, P]): PivotNotValues[T, TK, P] = + def pivot[P: CatalystPivotable]( + pivotColumn: TypedColumn[T, P] + ): PivotNotValues[T, TK, P] = PivotNotValues(self, groupedBy, pivotColumn) } private[ops] object AggregatingOps { + /** Utility function to help Spark with serialization of closures */ - def tuple1[K1, V, U](f: (K1, Iterator[V]) => U): (Tuple1[K1], Iterator[V]) => U = { - (x: Tuple1[K1], it: Iterator[V]) => f(x._1, it) + def tuple1[K1, V, U]( + f: (K1, Iterator[V]) => U + ): (Tuple1[K1], Iterator[V]) => U = { (x: Tuple1[K1], it: Iterator[V]) => + f(x._1, it) } } -/** Represents a typed Pivot operation. - */ +/** + * Represents a typed Pivot operation. + */ final case class Pivot[T, GroupedColumns <: HList, PivotType, Values <: HList]( - ds: TypedDataset[T], - groupedBy: GroupedColumns, - pivotedBy: TypedColumn[T, PivotType], - values: Values -) { + ds: TypedDataset[T], + groupedBy: GroupedColumns, + pivotedBy: TypedColumn[T, PivotType], + values: Values) { object agg extends ProductArgs { - def applyProduct[AggrColumns <: HList, AggrColumnTypes <: HList, GroupedColumnTypes <: HList, NumValues <: Nat, TypesForPivotedValues <: HList, TypesForPivotedValuesOpt <: HList, OutAsHList <: HList, Out] - (aggrColumns: AggrColumns) - (implicit + + def applyProduct[ + AggrColumns <: HList, + AggrColumnTypes <: HList, + GroupedColumnTypes <: HList, + NumValues <: Nat, + TypesForPivotedValues <: HList, + TypesForPivotedValuesOpt <: HList, + OutAsHList <: HList, + Out + ](aggrColumns: AggrColumns + )(implicit i0: AggregateTypes.Aux[T, AggrColumns, AggrColumnTypes], i1: ColumnTypes.Aux[T, GroupedColumns, GroupedColumnTypes], i2: Length.Aux[Values, NumValues], i3: Repeat.Aux[AggrColumnTypes, NumValues, TypesForPivotedValues], i4: Mapped.Aux[TypesForPivotedValues, Option, TypesForPivotedValuesOpt], - i5: Prepend.Aux[GroupedColumnTypes, TypesForPivotedValuesOpt, OutAsHList], + i5: Prepend.Aux[ + GroupedColumnTypes, + TypesForPivotedValuesOpt, + OutAsHList + ], i6: Tupler.Aux[OutAsHList, Out], i7: TypedEncoder[Out] ): TypedDataset[Out] = { - def mapAny[X](h: HList)(f: Any => X): List[X] = - h match { - case HNil => Nil - case x :: xs => f(x) :: mapAny(xs)(f) - } - - val aggCols: Seq[Column] = mapAny(aggrColumns)(x => FramelessInternals.column(x.asInstanceOf[TypedAggregate[_,_]].expr)) - val tmp = ds.dataset.toDF() - .groupBy(mapAny(groupedBy)(_.asInstanceOf[TypedColumn[_, _]].untyped): _*) - .pivot(pivotedBy.untyped.toString, mapAny(values)(identity)) - .agg(aggCols.head, aggCols.tail:_*) - .as[Out](TypedExpressionEncoder[Out]) - TypedDataset.create(tmp) - } + def mapAny[X](h: HList)(f: Any => X): List[X] = + h match { + case HNil => Nil + case x :: xs => f(x) :: mapAny(xs)(f) + } + + val aggCols: Seq[Column] = mapAny(aggrColumns)(x => + FramelessInternals.column(x.asInstanceOf[TypedAggregate[_, _]].expr) + ) + val tmp = ds.dataset + .toDF() + .groupBy( + mapAny(groupedBy)(_.asInstanceOf[TypedColumn[_, _]].untyped): _* + ) + .pivot(pivotedBy.untyped.toString, mapAny(values)(identity)) + .agg(aggCols.head, aggCols.tail: _*) + .as[Out](TypedExpressionEncoder[Out]) + TypedDataset.create(tmp) + } } } final case class PivotNotValues[T, GroupedColumns <: HList, PivotType]( - ds: TypedDataset[T], - groupedBy: GroupedColumns, - pivotedBy: TypedColumn[T, PivotType] -) extends ProductArgs { - - def onProduct[Values <: HList](values: Values)( - implicit validValues: ToList[Values, PivotType] // validValues: FilterNot.Aux[Values, PivotType, HNil] // did not work - ): Pivot[T, GroupedColumns, PivotType, Values] = Pivot(ds, groupedBy, pivotedBy, values) + ds: TypedDataset[T], + groupedBy: GroupedColumns, + pivotedBy: TypedColumn[T, PivotType]) + extends ProductArgs { + + def onProduct[Values <: HList]( + values: Values + )(implicit + validValues: ToList[ + Values, + PivotType + ] // validValues: FilterNot.Aux[Values, PivotType, HNil] // did not work + ): Pivot[T, GroupedColumns, PivotType, Values] = + Pivot(ds, groupedBy, pivotedBy, values) } diff --git a/dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala b/dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala index 81c221968..fcaf32cfd 100644 --- a/dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala +++ b/dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala @@ -2,25 +2,35 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.expressions.{Alias, CreateStruct} -import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{ Alias, CreateStruct } +import org.apache.spark.sql.catalyst.expressions.{ Expression, NamedExpression } import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{ LogicalPlan, Project } import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.types._ import org.apache.spark.sql.types.ObjectType import scala.reflect.ClassTag object FramelessInternals { - def objectTypeFor[A](implicit classTag: ClassTag[A]): ObjectType = ObjectType(classTag.runtimeClass) + + def objectTypeFor[A]( + implicit + classTag: ClassTag[A] + ): ObjectType = ObjectType(classTag.runtimeClass) def resolveExpr(ds: Dataset[_], colNames: Seq[String]): NamedExpression = { - ds.toDF().queryExecution.analyzed.resolve(colNames, ds.sparkSession.sessionState.analyzer.resolver).getOrElse { - throw new AnalysisException( - s"""Cannot resolve column name "$colNames" among (${ds.schema.fieldNames.mkString(", ")})""") - } + ds.toDF() + .queryExecution + .analyzed + .resolve(colNames, ds.sparkSession.sessionState.analyzer.resolver) + .getOrElse { + throw new AnalysisException( + s"""Cannot resolve column name "$colNames" among (${ds.schema.fieldNames + .mkString(", ")})""" + ) + } } /** Wraps a Catalyst [[Expression]] into a [[Column]]. */ @@ -38,18 +48,30 @@ object FramelessInternals { def getConf(ds: Dataset[_], key: String, default: String): String = ds.sqlContext.getConf(key, default) - def joinPlan(ds: Dataset[_], plan: LogicalPlan, leftPlan: LogicalPlan, rightPlan: LogicalPlan): LogicalPlan = { + 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) + Project( + List( + Alias(CreateStruct(leftOutput), "_1")(), + Alias(CreateStruct(rightOutput), "_2")() + ), + joined.analyzed + ) } - def mkDataset[T](source: Dataset[_], plan: LogicalPlan, encoder: Encoder[T]): Dataset[T] = + def mkDataset[T]( + source: Dataset[_], + plan: LogicalPlan, + encoder: Encoder[T] + ): Dataset[T] = new Dataset(source.sparkSession, plan, encoder) def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = @@ -57,34 +79,49 @@ object FramelessInternals { /** Builds an [[ExpressionEncoder]] from frameless' own serializer/deserializer expressions. */ def expressionEncoder[T]( - objSerializer: Expression, - objDeserializer: Expression, - classTag: ClassTag[T] - ): ExpressionEncoder[T] = + objSerializer: Expression, + objDeserializer: Expression, + classTag: ClassTag[T] + ): ExpressionEncoder[T] = new ExpressionEncoder[T](objSerializer, objDeserializer, classTag) // because org.apache.spark.sql.types.UserDefinedType is private[spark] - type UserDefinedType[A >: Null] = org.apache.spark.sql.types.UserDefinedType[A] + type UserDefinedType[A >: Null] = + org.apache.spark.sql.types.UserDefinedType[A] // below only tested in SelfJoinTests.colLeft and colRight are equivalent to col outside of joins // - via files (codegen) forces doGenCode eval. /** Expression to tag columns from the left hand side of join expression. */ - case class DisambiguateLeft[T](tagged: Expression) extends Expression with NonSQLExpression { + case class DisambiguateLeft[T](tagged: Expression) + extends Expression + with NonSQLExpression { def eval(input: InternalRow): Any = tagged.eval(input) def nullable: Boolean = false def children: Seq[Expression] = tagged :: Nil def dataType: DataType = tagged.dataType - protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = tagged.genCode(ctx) - protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(newChildren.head) + + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + tagged.genCode(ctx) + + protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression] + ): Expression = copy(newChildren.head) } /** Expression to tag columns from the right hand side of join expression. */ - case class DisambiguateRight[T](tagged: Expression) extends Expression with NonSQLExpression { + case class DisambiguateRight[T](tagged: Expression) + extends Expression + with NonSQLExpression { def eval(input: InternalRow): Any = tagged.eval(input) def nullable: Boolean = false def children: Seq[Expression] = tagged :: Nil def dataType: DataType = tagged.dataType - protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = tagged.genCode(ctx) - protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(newChildren.head) + + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + tagged.genCode(ctx) + + protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression] + ): Expression = copy(newChildren.head) } } diff --git a/dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala b/dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala index 81c221968..fcaf32cfd 100644 --- a/dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala +++ b/dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala @@ -2,25 +2,35 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.expressions.{Alias, CreateStruct} -import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{ Alias, CreateStruct } +import org.apache.spark.sql.catalyst.expressions.{ Expression, NamedExpression } import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{ LogicalPlan, Project } import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.types._ import org.apache.spark.sql.types.ObjectType import scala.reflect.ClassTag object FramelessInternals { - def objectTypeFor[A](implicit classTag: ClassTag[A]): ObjectType = ObjectType(classTag.runtimeClass) + + def objectTypeFor[A]( + implicit + classTag: ClassTag[A] + ): ObjectType = ObjectType(classTag.runtimeClass) def resolveExpr(ds: Dataset[_], colNames: Seq[String]): NamedExpression = { - ds.toDF().queryExecution.analyzed.resolve(colNames, ds.sparkSession.sessionState.analyzer.resolver).getOrElse { - throw new AnalysisException( - s"""Cannot resolve column name "$colNames" among (${ds.schema.fieldNames.mkString(", ")})""") - } + ds.toDF() + .queryExecution + .analyzed + .resolve(colNames, ds.sparkSession.sessionState.analyzer.resolver) + .getOrElse { + throw new AnalysisException( + s"""Cannot resolve column name "$colNames" among (${ds.schema.fieldNames + .mkString(", ")})""" + ) + } } /** Wraps a Catalyst [[Expression]] into a [[Column]]. */ @@ -38,18 +48,30 @@ object FramelessInternals { def getConf(ds: Dataset[_], key: String, default: String): String = ds.sqlContext.getConf(key, default) - def joinPlan(ds: Dataset[_], plan: LogicalPlan, leftPlan: LogicalPlan, rightPlan: LogicalPlan): LogicalPlan = { + 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) + Project( + List( + Alias(CreateStruct(leftOutput), "_1")(), + Alias(CreateStruct(rightOutput), "_2")() + ), + joined.analyzed + ) } - def mkDataset[T](source: Dataset[_], plan: LogicalPlan, encoder: Encoder[T]): Dataset[T] = + def mkDataset[T]( + source: Dataset[_], + plan: LogicalPlan, + encoder: Encoder[T] + ): Dataset[T] = new Dataset(source.sparkSession, plan, encoder) def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = @@ -57,34 +79,49 @@ object FramelessInternals { /** Builds an [[ExpressionEncoder]] from frameless' own serializer/deserializer expressions. */ def expressionEncoder[T]( - objSerializer: Expression, - objDeserializer: Expression, - classTag: ClassTag[T] - ): ExpressionEncoder[T] = + objSerializer: Expression, + objDeserializer: Expression, + classTag: ClassTag[T] + ): ExpressionEncoder[T] = new ExpressionEncoder[T](objSerializer, objDeserializer, classTag) // because org.apache.spark.sql.types.UserDefinedType is private[spark] - type UserDefinedType[A >: Null] = org.apache.spark.sql.types.UserDefinedType[A] + type UserDefinedType[A >: Null] = + org.apache.spark.sql.types.UserDefinedType[A] // below only tested in SelfJoinTests.colLeft and colRight are equivalent to col outside of joins // - via files (codegen) forces doGenCode eval. /** Expression to tag columns from the left hand side of join expression. */ - case class DisambiguateLeft[T](tagged: Expression) extends Expression with NonSQLExpression { + case class DisambiguateLeft[T](tagged: Expression) + extends Expression + with NonSQLExpression { def eval(input: InternalRow): Any = tagged.eval(input) def nullable: Boolean = false def children: Seq[Expression] = tagged :: Nil def dataType: DataType = tagged.dataType - protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = tagged.genCode(ctx) - protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(newChildren.head) + + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + tagged.genCode(ctx) + + protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression] + ): Expression = copy(newChildren.head) } /** Expression to tag columns from the right hand side of join expression. */ - case class DisambiguateRight[T](tagged: Expression) extends Expression with NonSQLExpression { + case class DisambiguateRight[T](tagged: Expression) + extends Expression + with NonSQLExpression { def eval(input: InternalRow): Any = tagged.eval(input) def nullable: Boolean = false def children: Seq[Expression] = tagged :: Nil def dataType: DataType = tagged.dataType - protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = tagged.genCode(ctx) - protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(newChildren.head) + + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + tagged.genCode(ctx) + + protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression] + ): Expression = copy(newChildren.head) } } diff --git a/dataset/src/main/spark-4/frameless/MapGroups.scala b/dataset/src/main/spark-4/frameless/MapGroups.scala index 6856acba4..25411420b 100644 --- a/dataset/src/main/spark-4/frameless/MapGroups.scala +++ b/dataset/src/main/spark-4/frameless/MapGroups.scala @@ -2,15 +2,19 @@ package frameless import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, MapGroups => SMapGroups} +import org.apache.spark.sql.catalyst.plans.logical.{ + LogicalPlan, + MapGroups => SMapGroups +} object MapGroups { + def apply[K: Encoder, T: Encoder, U: Encoder]( - func: (K, Iterator[T]) => TraversableOnce[U], - groupingAttributes: Seq[Attribute], - dataAttributes: Seq[Attribute], - child: LogicalPlan - ): LogicalPlan = + func: (K, Iterator[T]) => TraversableOnce[U], + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + child: LogicalPlan + ): LogicalPlan = SMapGroups( func, groupingAttributes, diff --git a/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala b/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala index 478c27457..fbedb954b 100644 --- a/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala +++ b/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala @@ -2,14 +2,19 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.expressions.{Alias, CreateStruct} -import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{ Alias, CreateStruct } +import org.apache.spark.sql.catalyst.expressions.{ Expression, NamedExpression } import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.JavaBeanEncoder import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.classic.{Dataset => ClassicDataset, SparkSession => ClassicSparkSession, ExpressionUtils, ColumnNodeToExpressionConverter} +import org.apache.spark.sql.catalyst.plans.logical.{ LogicalPlan, Project } +import org.apache.spark.sql.classic.{ + Dataset => ClassicDataset, + SparkSession => ClassicSparkSession, + ExpressionUtils, + ColumnNodeToExpressionConverter +} import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.types._ import org.apache.spark.sql.types.ObjectType @@ -25,7 +30,11 @@ import scala.reflect.ClassTag * own bridge between the two. */ object FramelessInternals { - def objectTypeFor[A](implicit classTag: ClassTag[A]): ObjectType = ObjectType(classTag.runtimeClass) + + def objectTypeFor[A]( + implicit + classTag: ClassTag[A] + ): ObjectType = ObjectType(classTag.runtimeClass) private def classic(ds: Dataset[_]): ClassicDataset[_] = ds.asInstanceOf[ClassicDataset[_]] @@ -53,7 +62,8 @@ object FramelessInternals { * disambiguation markers via `Expression.transform`, both of which require a real, traversable * expression tree - so convert the column's node eagerly instead. */ - def expr(column: Column): Expression = ColumnNodeToExpressionConverter(column.node) + def expr(column: Column): Expression = + ColumnNodeToExpressionConverter(column.node) def logicalPlan(ds: Dataset[_]): LogicalPlan = classic(ds).logicalPlan @@ -65,22 +75,37 @@ object FramelessInternals { def getConf(ds: Dataset[_], key: String, default: String): String = classic(ds).sparkSession.conf.get(key, default) - def joinPlan(ds: Dataset[_], plan: LogicalPlan, leftPlan: LogicalPlan, rightPlan: LogicalPlan): LogicalPlan = { + 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) + Project( + List( + Alias(CreateStruct(leftOutput), "_1")(), + Alias(CreateStruct(rightOutput), "_2")() + ), + joined.analyzed + ) } - def mkDataset[T](source: Dataset[_], plan: LogicalPlan, encoder: Encoder[T]): Dataset[T] = + def mkDataset[T]( + source: Dataset[_], + plan: LogicalPlan, + encoder: Encoder[T] + ): Dataset[T] = new ClassicDataset[T](classic(source).sparkSession, plan, encoder) def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = - ClassicDataset.ofRows(sparkSession.asInstanceOf[ClassicSparkSession], logicalPlan) + ClassicDataset.ofRows( + sparkSession.asInstanceOf[ClassicSparkSession], + logicalPlan + ) /** * Builds an [[ExpressionEncoder]] from frameless' own serializer/deserializer expressions. @@ -91,34 +116,53 @@ object FramelessInternals { * carrying the right `ClassTag` is therefore a correct, metadata-only stand-in. */ def expressionEncoder[T]( - objSerializer: Expression, - objDeserializer: Expression, - classTag: ClassTag[T] - ): ExpressionEncoder[T] = - new ExpressionEncoder[T](JavaBeanEncoder(classTag, Nil), objSerializer, objDeserializer) + objSerializer: Expression, + objDeserializer: Expression, + classTag: ClassTag[T] + ): ExpressionEncoder[T] = + new ExpressionEncoder[T]( + JavaBeanEncoder(classTag, Nil), + objSerializer, + objDeserializer + ) // because org.apache.spark.sql.types.UserDefinedType is private[spark] - type UserDefinedType[A >: Null] = org.apache.spark.sql.types.UserDefinedType[A] + type UserDefinedType[A >: Null] = + org.apache.spark.sql.types.UserDefinedType[A] // below only tested in SelfJoinTests.colLeft and colRight are equivalent to col outside of joins // - via files (codegen) forces doGenCode eval. /** Expression to tag columns from the left hand side of join expression. */ - case class DisambiguateLeft[T](tagged: Expression) extends Expression with NonSQLExpression { + case class DisambiguateLeft[T](tagged: Expression) + extends Expression + with NonSQLExpression { def eval(input: InternalRow): Any = tagged.eval(input) def nullable: Boolean = false def children: Seq[Expression] = tagged :: Nil def dataType: DataType = tagged.dataType - protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = tagged.genCode(ctx) - protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(newChildren.head) + + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + tagged.genCode(ctx) + + protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression] + ): Expression = copy(newChildren.head) } /** Expression to tag columns from the right hand side of join expression. */ - case class DisambiguateRight[T](tagged: Expression) extends Expression with NonSQLExpression { + case class DisambiguateRight[T](tagged: Expression) + extends Expression + with NonSQLExpression { def eval(input: InternalRow): Any = tagged.eval(input) def nullable: Boolean = false def children: Seq[Expression] = tagged :: Nil def dataType: DataType = tagged.dataType - protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = tagged.genCode(ctx) - protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = copy(newChildren.head) + + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + tagged.genCode(ctx) + + protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression] + ): Expression = copy(newChildren.head) } } diff --git a/dataset/src/test/scala/frameless/SchemaTests.scala b/dataset/src/test/scala/frameless/SchemaTests.scala index 7dac7215a..89fed7f86 100644 --- a/dataset/src/test/scala/frameless/SchemaTests.scala +++ b/dataset/src/test/scala/frameless/SchemaTests.scala @@ -2,7 +2,7 @@ package frameless import frameless.functions.aggregate._ import frameless.functions._ -import org.apache.spark.sql.types.{Metadata, StructType} +import org.apache.spark.sql.types.{ Metadata, StructType } import org.scalacheck.Prop import org.scalacheck.Prop._ import org.scalatest.matchers.should.Matchers @@ -12,10 +12,17 @@ class SchemaTests extends TypedDatasetSuite with Matchers { def structToNonNullable(struct: StructType): StructType = { // Spark 4 attaches metadata to aggregate output columns; clear it (and nullability) // so comparisons consider only field names and types. No-op on Spark 3.x. - StructType(struct.fields.map( f => f.copy(nullable = false, metadata = Metadata.empty))) + StructType( + struct.fields.map(f => + f.copy(nullable = false, metadata = Metadata.empty) + ) + ) } - def prop[A](dataset: TypedDataset[A], ignoreNullable: Boolean = false): Prop = { + def prop[A]( + dataset: TypedDataset[A], + ignoreNullable: Boolean = false + ): Prop = { val schema = dataset.dataset.schema Prop.all( @@ -26,7 +33,9 @@ class SchemaTests extends TypedDatasetSuite with Matchers { if (!ignoreNullable) TypedExpressionEncoder.targetStructType(dataset.encoder) ?= schema else - structToNonNullable(TypedExpressionEncoder.targetStructType(dataset.encoder)) ?= structToNonNullable(schema) + structToNonNullable( + TypedExpressionEncoder.targetStructType(dataset.encoder) + ) ?= structToNonNullable(schema) ) } diff --git a/dataset/src/test/scala/frameless/TypedDatasetSuite.scala b/dataset/src/test/scala/frameless/TypedDatasetSuite.scala index 1314fe74e..e31be7cbc 100644 --- a/dataset/src/test/scala/frameless/TypedDatasetSuite.scala +++ b/dataset/src/test/scala/frameless/TypedDatasetSuite.scala @@ -2,28 +2,35 @@ package frameless import com.globalmentor.apache.hadoop.fs.BareLocalFileSystem import org.apache.hadoop.fs.local.StreamingFS -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.{SQLContext, SparkSession} +import org.apache.spark.{ SparkConf, SparkContext } +import org.apache.spark.sql.{ SQLContext, SparkSession } import org.scalactic.anyvals.PosZInt import org.scalatest.BeforeAndAfterAll import org.scalatestplus.scalacheck.Checkers import org.scalacheck.Prop import org.scalacheck.Prop._ -import scala.util.{Properties, Try} +import scala.util.{ Properties, Try } import org.scalatest.funsuite.AnyFunSuite trait SparkTesting { self: BeforeAndAfterAll => - val appID: String = new java.util.Date().toString + math.floor(math.random * 10E4).toLong.toString + val appID: String = new java.util.Date().toString + math + .floor(math.random * 10e4) + .toLong + .toString /** * Allows bare naked to be used instead of winutils for testing / dev */ def registerFS(sparkConf: SparkConf): SparkConf = { if (System.getProperty("os.name").startsWith("Windows")) - sparkConf.set("spark.hadoop.fs.file.impl", classOf[BareLocalFileSystem].getName). - set("spark.hadoop.fs.AbstractFileSystem.file.impl", classOf[StreamingFS].getName) + sparkConf + .set("spark.hadoop.fs.file.impl", classOf[BareLocalFileSystem].getName) + .set( + "spark.hadoop.fs.AbstractFileSystem.file.impl", + classOf[StreamingFS].getName + ) else sparkConf } @@ -45,9 +52,9 @@ trait SparkTesting { self: BeforeAndAfterAll => implicit def sc: SparkContext = session.sparkContext implicit def sqlContext: SQLContext = session.sqlContext - def registerOptimizations(sqlContext: SQLContext): Unit = { } + def registerOptimizations(sqlContext: SQLContext): Unit = {} - def addSparkConfigProperties(config: SparkConf): Unit = { } + def addSparkConfigProperties(config: SparkConf): Unit = {} override def beforeAll(): Unit = { assert(s == null) @@ -64,11 +71,16 @@ trait SparkTesting { self: BeforeAndAfterAll => } } +class TypedDatasetSuite + extends AnyFunSuite + with Checkers + with BeforeAndAfterAll + with SparkTesting { -class TypedDatasetSuite extends AnyFunSuite with Checkers with BeforeAndAfterAll with SparkTesting { // Limit size of generated collections and number of checks to avoid OutOfMemoryError implicit override val generatorDrivenConfig: PropertyCheckConfiguration = { - def getPosZInt(name: String, default: PosZInt) = Properties.envOrNone(s"FRAMELESS_GEN_${name}") + def getPosZInt(name: String, default: PosZInt) = Properties + .envOrNone(s"FRAMELESS_GEN_${name}") .flatMap(s => Try(s.toInt).toOption) .flatMap(PosZInt.from) .getOrElse(default) @@ -80,17 +92,24 @@ class TypedDatasetSuite extends AnyFunSuite with Checkers with BeforeAndAfterAll implicit val sparkDelay: SparkDelay[Job] = Job.framelessSparkDelayForJob - def approximatelyEqual[A](a: A, b: A)(implicit numeric: Numeric[A]): Prop = { + def approximatelyEqual[A]( + a: A, + b: A + )(implicit + numeric: Numeric[A] + ): Prop = { val da = numeric.toDouble(a) val db = numeric.toDouble(b) - val epsilon = 1E-6 + val epsilon = 1e-6 // Spark has a weird behaviour concerning expressions that should return Inf // Most of the time they return NaN instead, for instance stddev of Seq(-7.827553978923477E227, -5.009124275715786E153) - if((da.isNaN || da.isInfinity) && (db.isNaN || db.isInfinity)) proved + if ((da.isNaN || da.isInfinity) && (db.isNaN || db.isInfinity)) proved else if ( (da - db).abs < epsilon || - (da - db).abs < da.abs / 100) - proved - else falsified :| s"Expected $a but got $b, which is more than 1% off and greater than epsilon = $epsilon." + (da - db).abs < da.abs / 100 + ) + proved + else + falsified :| s"Expected $a but got $b, which is more than 1% off and greater than epsilon = $epsilon." } } diff --git a/dataset/src/test/scala/frameless/forward/SQLContextTests.scala b/dataset/src/test/scala/frameless/forward/SQLContextTests.scala index 454dca8a2..06c14f651 100644 --- a/dataset/src/test/scala/frameless/forward/SQLContextTests.scala +++ b/dataset/src/test/scala/frameless/forward/SQLContextTests.scala @@ -1,14 +1,15 @@ package frameless import org.scalacheck.Prop -import org.scalacheck.Prop.{forAll, _} +import org.scalacheck.Prop.{ forAll, _ } class SQLContextTests extends TypedDatasetSuite { test("sqlContext") { def prop[A: TypedEncoder](data: Vector[A]): Prop = { val dataset = TypedDataset.create[A](data) - dataset.sqlContext =? org.apache.spark.sql.FramelessInternals.sqlContext(dataset.dataset) + dataset.sqlContext =? org.apache.spark.sql.FramelessInternals + .sqlContext(dataset.dataset) } check(forAll(prop[Int] _)) diff --git a/dataset/src/test/spark-3.3+/frameless/sql/rules/FramelessLitPushDownTests.scala b/dataset/src/test/spark-3.3+/frameless/sql/rules/FramelessLitPushDownTests.scala index 5b09f1cc3..1df361b9b 100644 --- a/dataset/src/test/spark-3.3+/frameless/sql/rules/FramelessLitPushDownTests.scala +++ b/dataset/src/test/spark-3.3+/frameless/sql/rules/FramelessLitPushDownTests.scala @@ -3,7 +3,7 @@ package frameless.sql.rules import frameless._ import frameless.functions.Lit import org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToInstant -import org.apache.spark.sql.sources.{EqualTo, GreaterThanOrEqual, IsNotNull} +import org.apache.spark.sql.sources.{ EqualTo, GreaterThanOrEqual, IsNotNull } import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import java.time.Instant @@ -15,7 +15,8 @@ class FramelessLitPushDownTests extends SQLRulesSuite { test("java.sql.Timestamp push-down") { val expected = java.sql.Timestamp.from(microsToInstant(now)) val expectedStructure = X1(SQLTimestamp(now)) - val expectedPushDownFilters = List(IsNotNull("a"), GreaterThanOrEqual("a", expected)) + val expectedPushDownFilters = + List(IsNotNull("a"), GreaterThanOrEqual("a", expected)) predicatePushDownTest[SQLTimestamp]( expectedStructure, @@ -28,7 +29,8 @@ class FramelessLitPushDownTests extends SQLRulesSuite { test("java.time.Instant push-down") { val expected = java.sql.Timestamp.from(microsToInstant(now)) val expectedStructure = X1(microsToInstant(now)) - val expectedPushDownFilters = List(IsNotNull("a"), GreaterThanOrEqual("a", expected)) + val expectedPushDownFilters = + List(IsNotNull("a"), GreaterThanOrEqual("a", expected)) predicatePushDownTest[Instant]( expectedStructure, @@ -41,7 +43,10 @@ class FramelessLitPushDownTests extends SQLRulesSuite { test("struct push-down") { type Payload = X4[Int, Int, Int, Int] val expectedStructure = X1(X4(1, 2, 3, 4)) - val expected = new GenericRowWithSchema(Array(1, 2, 3, 4), TypedExpressionEncoder[Payload].schema) + val expected = new GenericRowWithSchema( + Array(1, 2, 3, 4), + TypedExpressionEncoder[Payload].schema + ) val expectedPushDownFilters = List(IsNotNull("a"), EqualTo("a", expected)) predicatePushDownTest[Payload]( From 50b78881b063d98aec7eca6a00b4df999ee7aab4 Mon Sep 17 00:00:00 2001 From: Marios Iliofotou Date: Tue, 19 May 2026 22:48:21 -0700 Subject: [PATCH 3/7] Make docs/mdoc build on JDK 17 (site CI job) Adding a JDK 17 CI leg for Spark 4 made sbt-typelevel run the Generate Site job on JDK 17 (it picks the last configured Java). mdoc executes Spark code, which needs the module --add-opens flags on JDK 17. Fork the docs run, pass the flags through (extracted into sparkJava17Options, shared with the test config), and pin the forked run's working directory to the repo root so docs keep finding their relative data files (docs/iris.data). Co-Authored-By: Claude Opus 4.7 (1M context) --- build.sbt | 53 ++++++++++++++++++++++++++++++----------------------- 1 file changed, 30 insertions(+), 23 deletions(-) diff --git a/build.sbt b/build.sbt index a47bc51c6..511414fa4 100644 --- a/build.sbt +++ b/build.sbt @@ -248,7 +248,14 @@ lazy val docs = project "org.typelevel" % "kind-projector" % "0.13.4" cross CrossVersion.full ), scalacOptions += "-Ydelambdafy:inline", - libraryDependencies += "org.typelevel" %% "mouse" % "1.3.2" + libraryDependencies += "org.typelevel" %% "mouse" % "1.3.2", + // mdoc executes Spark code via `Compile / runMain`; on JDK 17 (the site CI job) Spark + // needs the module --add-opens flags, so fork the run and pass them through. Forking + // changes the working directory, so pin it to the repo root where the docs read their + // relative data files (e.g. docs/iris.data). + Compile / run / fork := true, + Compile / run / javaOptions ++= sparkJava17Options, + Compile / run / baseDirectory := (LocalRootProject / baseDirectory).value ) .dependsOn(dataset, cats, ml) @@ -361,6 +368,27 @@ lazy val scalacOptionSettings = Def.setting { baseScalacOptions(scalaVersion.value) } +// JVM flags Spark needs on JDK 17+ (the module system blocks its reflective access +// to java.base internals otherwise). Empty on JDK 8/11. Reused by tests and the docs run. +lazy val sparkJava17Options: Seq[String] = + if (sys.props("java.specification.version").toDouble >= 17.0) { + Seq( + "--add-opens=java.base/java.lang=ALL-UNNAMED", + "--add-opens=java.base/java.lang.invoke=ALL-UNNAMED", + "--add-opens=java.base/java.lang.reflect=ALL-UNNAMED", + "--add-opens=java.base/java.io=ALL-UNNAMED", + "--add-opens=java.base/java.net=ALL-UNNAMED", + "--add-opens=java.base/java.nio=ALL-UNNAMED", + "--add-opens=java.base/java.util=ALL-UNNAMED", + "--add-opens=java.base/java.util.concurrent=ALL-UNNAMED", + "--add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED", + "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", + "--add-opens=java.base/sun.nio.cs=ALL-UNNAMED", + "--add-opens=java.base/sun.security.action=ALL-UNNAMED", + "--add-opens=java.base/sun.util.calendar=ALL-UNNAMED" + ) + } else Seq.empty + lazy val framelessSettings = Seq( scalacOptions ++= scalacOptionSettings.value, Test / testOptions += Tests.Argument(TestFrameworks.ScalaTest, "-oDF"), @@ -370,28 +398,7 @@ lazy val framelessSettings = Seq( "org.scalatestplus" %% "scalatestplus-scalacheck" % scalatestplus % Test, "org.scalacheck" %% "scalacheck" % scalacheck % Test ), - Test / javaOptions ++= { - val baseOptions = Seq("-Xmx1G", "-ea") - val java17Options = - if (sys.props("java.specification.version").toDouble >= 17.0) { - Seq( - "--add-opens=java.base/java.lang=ALL-UNNAMED", - "--add-opens=java.base/java.lang.invoke=ALL-UNNAMED", - "--add-opens=java.base/java.lang.reflect=ALL-UNNAMED", - "--add-opens=java.base/java.io=ALL-UNNAMED", - "--add-opens=java.base/java.net=ALL-UNNAMED", - "--add-opens=java.base/java.nio=ALL-UNNAMED", - "--add-opens=java.base/java.util=ALL-UNNAMED", - "--add-opens=java.base/java.util.concurrent=ALL-UNNAMED", - "--add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED", - "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", - "--add-opens=java.base/sun.nio.cs=ALL-UNNAMED", - "--add-opens=java.base/sun.security.action=ALL-UNNAMED", - "--add-opens=java.base/sun.util.calendar=ALL-UNNAMED" - ) - } else Seq.empty - baseOptions ++ java17Options - }, + Test / javaOptions ++= Seq("-Xmx1G", "-ea") ++ sparkJava17Options, Test / fork := true, Test / parallelExecution := false, mimaPreviousArtifacts ~= { From 332dd6f22b3413e0d97aaf037255a8f3bb88fd44 Mon Sep 17 00:00:00 2001 From: Marios Iliofotou Date: Tue, 19 May 2026 23:04:40 -0700 Subject: [PATCH 4/7] Add MiMa filters for FramelessInternals compat-seam changes The Spark 4 port reworked FramelessInternals (internal version-compat plumbing, not intended public API): `column` is now the Expression->Column bridge and `mkDataset` derives the session from the source Dataset instead of taking a SQLContext. Both are binary-incompatible signature changes flagged by MiMa against the 3.x baselines (0.14.0/0.14.1), so exclude them. Co-Authored-By: Claude Opus 4.7 (1M context) --- build.sbt | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 511414fa4..19e9d5d5b 100644 --- a/build.sbt +++ b/build.sbt @@ -305,7 +305,13 @@ lazy val datasetSettings = mc("frameless.functions.FramelessLit"), mc(f"frameless.functions.FramelessLit$$"), dmm("frameless.functions.package.litAggr"), - dmm("org.apache.spark.sql.FramelessInternals.column") + dmm("org.apache.spark.sql.FramelessInternals.column"), + // FramelessInternals is internal plumbing (Spark-version compat seam), not part of + // the intended public API. Spark 4 required reworking it: `column` is now the + // Expression->Column bridge and `mkDataset` derives the session from the source + // Dataset instead of taking a SQLContext. + imt("org.apache.spark.sql.FramelessInternals.column"), + imt("org.apache.spark.sql.FramelessInternals.mkDataset") ) }, coverageExcludedPackages := "org.apache.spark.sql.reflection", From 984da11c85f459556422527b5318fa80bfd31002 Mon Sep 17 00:00:00 2001 From: Marios Iliofotou Date: Tue, 19 May 2026 23:20:17 -0700 Subject: [PATCH 5/7] Fix Scala 2.12 scaladoc: use backticks instead of [[]] links Scala 2.12's scaladoc fails (fatally) on [[Expression]] / [[Column]] / [[ExpressionEncoder]] doc links in FramelessInternals because those Spark types aren't resolvable in the doc scope. Use backticks (code spans) instead. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../org/apache/spark/sql/FramelessInternals.scala | 4 ++-- .../spark-3/org/apache/spark/sql/FramelessInternals.scala | 4 ++-- .../spark-4/org/apache/spark/sql/FramelessInternals.scala | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala b/dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala index fcaf32cfd..3022bf23c 100644 --- a/dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala +++ b/dataset/src/main/spark-3.4+/org/apache/spark/sql/FramelessInternals.scala @@ -33,7 +33,7 @@ object FramelessInternals { } } - /** Wraps a Catalyst [[Expression]] into a [[Column]]. */ + /** Wraps a Catalyst `Expression` into a `Column`. */ def column(e: Expression): Column = new Column(e) def expr(column: Column): Expression = column.expr @@ -77,7 +77,7 @@ object FramelessInternals { def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = Dataset.ofRows(sparkSession, logicalPlan) - /** Builds an [[ExpressionEncoder]] from frameless' own serializer/deserializer expressions. */ + /** Builds an `ExpressionEncoder` from frameless' own serializer/deserializer expressions. */ def expressionEncoder[T]( objSerializer: Expression, objDeserializer: Expression, diff --git a/dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala b/dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala index fcaf32cfd..3022bf23c 100644 --- a/dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala +++ b/dataset/src/main/spark-3/org/apache/spark/sql/FramelessInternals.scala @@ -33,7 +33,7 @@ object FramelessInternals { } } - /** Wraps a Catalyst [[Expression]] into a [[Column]]. */ + /** Wraps a Catalyst `Expression` into a `Column`. */ def column(e: Expression): Column = new Column(e) def expr(column: Column): Expression = column.expr @@ -77,7 +77,7 @@ object FramelessInternals { def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = Dataset.ofRows(sparkSession, logicalPlan) - /** Builds an [[ExpressionEncoder]] from frameless' own serializer/deserializer expressions. */ + /** Builds an `ExpressionEncoder` from frameless' own serializer/deserializer expressions. */ def expressionEncoder[T]( objSerializer: Expression, objDeserializer: Expression, diff --git a/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala b/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala index fbedb954b..6daf2b4e4 100644 --- a/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala +++ b/dataset/src/main/spark-4/org/apache/spark/sql/FramelessInternals.scala @@ -51,11 +51,11 @@ object FramelessInternals { } } - /** Wraps a Catalyst [[Expression]] into a [[Column]] (Spark 4 bridge). */ + /** Wraps a Catalyst `Expression` into a `Column` (Spark 4 bridge). */ def column(e: Expression): Column = ExpressionUtils.column(e) /** - * Extracts the Catalyst [[Expression]] from a [[Column]]. + * Extracts the Catalyst `Expression` from a `Column`. * * `ExpressionUtils.expression` would return a lazy `ColumnNodeExpression` wrapper, which is * `Unevaluable` and exposes no children. frameless builds join plans by hand and rewrites @@ -108,7 +108,7 @@ object FramelessInternals { ) /** - * Builds an [[ExpressionEncoder]] from frameless' own serializer/deserializer expressions. + * Builds an `ExpressionEncoder` from frameless' own serializer/deserializer expressions. * * Spark 4's `ExpressionEncoder` takes a leading `AgnosticEncoder` (SPARK-49025), but it is * only read for its `clsTag` and an Option-wrapping check - the serializer, deserializer and From 4f289db3a06494c0990560730dc05048280302f2 Mon Sep 17 00:00:00 2001 From: Marios Iliofotou Date: Wed, 20 May 2026 08:04:48 -0700 Subject: [PATCH 6/7] Add value-level self-join regression test The existing self-join tests only compare row counts. This collects and verifies the decoded (T, U) tuples through the colLeft/colRight disambiguation path - a regression guard for the Spark 4 ColumnNode rework, which broke that path (only count-level coverage would have missed a subtly wrong decode). Passes unchanged on Spark 3.x. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../test/scala/frameless/SelfJoinTests.scala | 249 ++++++++++++------ 1 file changed, 166 insertions(+), 83 deletions(-) diff --git a/dataset/src/test/scala/frameless/SelfJoinTests.scala b/dataset/src/test/scala/frameless/SelfJoinTests.scala index cede7be2a..742429108 100644 --- a/dataset/src/test/scala/frameless/SelfJoinTests.scala +++ b/dataset/src/test/scala/frameless/SelfJoinTests.scala @@ -2,13 +2,18 @@ package frameless import org.scalacheck.Prop import org.scalacheck.Prop._ -import org.apache.spark.sql.{SparkSession, functions => sparkFunctions} +import org.apache.spark.sql.{ SparkSession, functions => sparkFunctions } class SelfJoinTests extends TypedDatasetSuite { + // Without crossJoin.enabled=true Spark doesn't like trivial join conditions: // [error] Join condition is missing or trivial. // [error] Use the CROSS JOIN syntax to allow cartesian products between these relations. - def allowTrivialJoin[T](body: => T)(implicit session: SparkSession): T = { + def allowTrivialJoin[T]( + body: => T + )(implicit + session: SparkSession + ): T = { val crossJoin = "spark.sql.crossJoin.enabled" val oldSetting = session.conf.get(crossJoin) session.conf.set(crossJoin, "true") @@ -17,7 +22,11 @@ class SelfJoinTests extends TypedDatasetSuite { result } - def allowAmbiguousJoin[T](body: => T)(implicit session: SparkSession): T = { + def allowAmbiguousJoin[T]( + body: => T + )(implicit + session: SparkSession + ): T = { val crossJoin = "spark.sql.analyzer.failAmbiguousSelfJoin" val oldSetting = session.conf.get(crossJoin) session.conf.set(crossJoin, "false") @@ -27,22 +36,26 @@ class SelfJoinTests extends TypedDatasetSuite { } test("self join with colLeft/colRight disambiguation") { - def prop[ - A : TypedEncoder : Ordering, - B : TypedEncoder : Ordering - ](dx: List[X2[A, B]], d: X2[A, B]): Prop = allowAmbiguousJoin { + def prop[A: TypedEncoder: Ordering, B: TypedEncoder: Ordering]( + dx: List[X2[A, B]], + d: X2[A, B] + ): Prop = allowAmbiguousJoin { val data = d :: dx val ds = TypedDataset.create(data) // This is the way to write unambiguous self-join in vanilla, see https://goo.gl/XnkSUD val df1 = ds.dataset.as("df1") val df2 = ds.dataset.as("df2") - val vanilla = df1.join(df2, - sparkFunctions.col("df1.a") === sparkFunctions.col("df2.a")).count() + val vanilla = df1 + .join(df2, sparkFunctions.col("df1.a") === sparkFunctions.col("df2.a")) + .count() - val typed = ds.joinInner(ds)( - ds.colLeft('a) === ds.colRight('a) - ).count().run() + val typed = ds + .joinInner(ds)( + ds.colLeft('a) === ds.colRight('a) + ) + .count() + .run() vanilla ?= typed } @@ -50,48 +63,92 @@ class SelfJoinTests extends TypedDatasetSuite { check(prop[Int, Int] _) } + test("self join collects correct values via colLeft/colRight") { + def prop[A: TypedEncoder: Ordering, B: TypedEncoder: Ordering]( + dx: List[X2[A, B]], + d: X2[A, B] + ): Prop = allowAmbiguousJoin { + val data = d :: dx + val ds = TypedDataset.create(data) + + // Collecting the joined tuples exercises the colLeft/colRight disambiguation and the + // (T, U) ExpressionEncoder end to end, not just the row count: a regression guard for + // Spark 4, where columns no longer wrap Catalyst expressions directly. + val typed = ds + .joinInner(ds)(ds.colLeft('a) === ds.colRight('a)) + .collect() + .run() + .toVector + .sorted + + val expected = (for { + l <- data + r <- data + if l.a == r.a + } yield (l, r)).toVector.sorted + + typed ?= expected + } + + check(prop[Int, Int] _) + check(prop[String, Long] _) + } + test("trivial self join") { - def prop[ - A : TypedEncoder : Ordering, - B : TypedEncoder : Ordering - ](dx: List[X2[A, B]], d: X2[A, B]): Prop = - allowTrivialJoin { allowAmbiguousJoin { - - val data = d :: dx - val ds = TypedDataset.create(data) - val untyped = ds.dataset - // Interestingly, even with aliasing it seems that it's impossible to - // obtain a trivial join condition of shape df1.a == df1.a, Spark we - // always interpret that as df1.a == df2.a. For the purpose of this - // test we fall-back to lit(true) instead. - // val trivial = sparkFunctions.col("df1.a") === sparkFunctions.col("df1.a") - val trivial = sparkFunctions.lit(true) - val vanilla = untyped.as("df1").join(untyped.as("df2"), trivial).count() - - val typed = ds.joinInner(ds)(ds.colLeft('a) === ds.colLeft('a)).count().run - vanilla ?= typed - } } + def prop[A: TypedEncoder: Ordering, B: TypedEncoder: Ordering]( + dx: List[X2[A, B]], + d: X2[A, B] + ): Prop = + allowTrivialJoin { + allowAmbiguousJoin { + + val data = d :: dx + val ds = TypedDataset.create(data) + val untyped = ds.dataset + // Interestingly, even with aliasing it seems that it's impossible to + // obtain a trivial join condition of shape df1.a == df1.a, Spark we + // always interpret that as df1.a == df2.a. For the purpose of this + // test we fall-back to lit(true) instead. + // val trivial = sparkFunctions.col("df1.a") === sparkFunctions.col("df1.a") + val trivial = sparkFunctions.lit(true) + val vanilla = + untyped.as("df1").join(untyped.as("df2"), trivial).count() + + val typed = + ds.joinInner(ds)(ds.colLeft('a) === ds.colLeft('a)).count().run + vanilla ?= typed + } + } check(prop[Int, Int] _) } test("self join with unambiguous expression") { def prop[ - A : TypedEncoder : CatalystNumeric : Ordering, - B : TypedEncoder : Ordering - ](data: List[X3[A, A, B]]): Prop = allowAmbiguousJoin { + A: TypedEncoder: CatalystNumeric: Ordering, + B: TypedEncoder: Ordering + ](data: List[X3[A, A, B]] + ): Prop = allowAmbiguousJoin { val ds = TypedDataset.create(data) val df1 = ds.dataset.alias("df1") val df2 = ds.dataset.alias("df2") - val vanilla = df1.join(df2, - (sparkFunctions.col("df1.a") + sparkFunctions.col("df1.b")) === - (sparkFunctions.col("df2.a") + sparkFunctions.col("df2.b"))).count() - - val typed = ds.joinInner(ds)( - (ds.colLeft('a) + ds.colLeft('b)) === (ds.colRight('a) + ds.colRight('b)) - ).count().run() + val vanilla = df1 + .join( + df2, + (sparkFunctions.col("df1.a") + sparkFunctions.col("df1.b")) === + (sparkFunctions.col("df2.a") + sparkFunctions.col("df2.b")) + ) + .count() + + val typed = ds + .joinInner(ds)( + (ds.colLeft('a) + ds.colLeft('b)) === (ds.colRight('a) + ds + .colRight('b)) + ) + .count() + .run() vanilla ?= typed } @@ -99,41 +156,57 @@ class SelfJoinTests extends TypedDatasetSuite { check(prop[Int, Int] _) } - test("Do you want ambiguous self join? This is how you get ambiguous self join.") { + test( + "Do you want ambiguous self join? This is how you get ambiguous self join." + ) { def prop[ - A : TypedEncoder : CatalystNumeric : Ordering, - B : TypedEncoder : Ordering - ](data: List[X3[A, A, B]]): Prop = - allowTrivialJoin { allowAmbiguousJoin { - val ds = TypedDataset.create(data) - - // The point I'm making here is that it "behaves just like Spark". I - // don't know (or really care about how) how Spark disambiguates that - // internally... - val vanilla = ds.dataset.join(ds.dataset, - (ds.dataset("a") + ds.dataset("b")) === - (ds.dataset("a") + ds.dataset("b"))).count() - - val typed = ds.joinInner(ds)( - (ds.col('a) + ds.col('b)) === (ds.col('a) + ds.col('b)) - ).count().run() - - vanilla ?= typed - } } - - check(prop[Int, Int] _) - } + A: TypedEncoder: CatalystNumeric: Ordering, + B: TypedEncoder: Ordering + ](data: List[X3[A, A, B]] + ): Prop = + allowTrivialJoin { + allowAmbiguousJoin { + val ds = TypedDataset.create(data) + + // The point I'm making here is that it "behaves just like Spark". I + // don't know (or really care about how) how Spark disambiguates that + // internally... + val vanilla = ds.dataset + .join( + ds.dataset, + (ds.dataset("a") + ds.dataset("b")) === + (ds.dataset("a") + ds.dataset("b")) + ) + .count() + + val typed = ds + .joinInner(ds)( + (ds.col('a) + ds.col('b)) === (ds.col('a) + ds.col('b)) + ) + .count() + .run() + + vanilla ?= typed + } + } + + check(prop[Int, Int] _) + } test("colLeft and colRight are equivalent to col outside of joins") { - def prop[A, B, C, D](data: Vector[X4[A, B, C, D]])( - implicit - ea: TypedEncoder[A], - ex4: TypedEncoder[X4[A, B, C, D]] - ): Prop = { + def prop[A, B, C, D]( + data: Vector[X4[A, B, C, D]] + )(implicit + ea: TypedEncoder[A], + ex4: TypedEncoder[X4[A, B, C, D]] + ): Prop = { val dataset = TypedDataset.create(data) - val selectedCol = dataset.select(dataset.col [A]('a)).collect().run().toVector - val selectedColLeft = dataset.select(dataset.colLeft [A]('a)).collect().run().toVector - val selectedColRight = dataset.select(dataset.colRight[A]('a)).collect().run().toVector + val selectedCol = + dataset.select(dataset.col[A]('a)).collect().run().toVector + val selectedColLeft = + dataset.select(dataset.colLeft[A]('a)).collect().run().toVector + val selectedColRight = + dataset.select(dataset.colRight[A]('a)).collect().run().toVector (selectedCol ?= selectedColLeft) && (selectedCol ?= selectedColRight) } @@ -145,16 +218,26 @@ class SelfJoinTests extends TypedDatasetSuite { } test("colLeft and colRight are equivalent to col outside of joins - via files (codegen)") { - def prop[A, B, C, D](data: Vector[X4[A, B, C, D]])( - implicit - ea: TypedEncoder[A], - ex4: TypedEncoder[X4[A, B, C, D]] - ): Prop = { - TypedDataset.create(data).write.mode("overwrite").parquet("./target/testData") - val dataset = TypedDataset.createUnsafe[X4[A, B, C, D]](session.read.parquet("./target/testData")) - val selectedCol = dataset.select(dataset.col [A]('a)).collect().run().toVector - val selectedColLeft = dataset.select(dataset.colLeft [A]('a)).collect().run().toVector - val selectedColRight = dataset.select(dataset.colRight[A]('a)).collect().run().toVector + def prop[A, B, C, D]( + data: Vector[X4[A, B, C, D]] + )(implicit + ea: TypedEncoder[A], + ex4: TypedEncoder[X4[A, B, C, D]] + ): Prop = { + TypedDataset + .create(data) + .write + .mode("overwrite") + .parquet("./target/testData") + val dataset = TypedDataset.createUnsafe[X4[A, B, C, D]]( + session.read.parquet("./target/testData") + ) + val selectedCol = + dataset.select(dataset.col[A]('a)).collect().run().toVector + val selectedColLeft = + dataset.select(dataset.colLeft[A]('a)).collect().run().toVector + val selectedColRight = + dataset.select(dataset.colRight[A]('a)).collect().run().toVector (selectedCol ?= selectedColLeft) && (selectedCol ?= selectedColRight) } From ce67c0853fd44d907138a4a36449751ed6c90565 Mon Sep 17 00:00:00 2001 From: Marios Iliofotou Date: Wed, 20 May 2026 10:20:54 -0700 Subject: [PATCH 7/7] Keep imports closer to source in TypedExpressionEncoder Revert the opinionated merge of the standalone `import ...Encoder` into a braced group; add FramelessInternals as a separate plain import instead. scalafmt does not merge imports, so this stays linter-clean while staying closer to the original source. Co-Authored-By: Claude Opus 4.7 (1M context) --- dataset/src/main/scala/frameless/TypedExpressionEncoder.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala b/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala index ba51a6ef3..71fa286a5 100644 --- a/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala +++ b/dataset/src/main/scala/frameless/TypedExpressionEncoder.scala @@ -1,6 +1,7 @@ package frameless -import org.apache.spark.sql.{ Encoder, FramelessInternals } +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.FramelessInternals import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal import org.apache.spark.sql.catalyst.expressions.{ BoundReference,