[spark] 03/03: Refine solution
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git commit 890748873bd8bd72b34d3f907ecdb72a694234c9 Author: Herman van Hovell AuthorDate: Mon Aug 14 05:32:57 2023 +0200 Refine solution --- .../spark/sql/catalyst/encoders/OuterScopes.scala | 49 +- .../expressions/codegen/CodeGenerator.scala| 18 ++-- 2 files changed, 33 insertions(+), 34 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala index c2ac504c846..6c10e8ece80 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala @@ -26,28 +26,9 @@ import org.apache.spark.util.SparkClassUtils object OuterScopes { private[this] val queue = new ReferenceQueue[AnyRef] - private class HashableWeakReference(v: AnyRef) extends WeakReference[AnyRef](v, queue) { -private[this] val hash = v.hashCode() -override def hashCode(): Int = hash -override def equals(obj: Any): Boolean = { - obj match { -case other: HashableWeakReference => - // Note that referential equality is used to identify & purge - // references from the map whose' referent went out of scope. - if (this eq other) { -true - } else { -val referent = get() -val otherReferent = other.get() -referent != null && otherReferent != null && Objects.equals(referent, otherReferent) - } -case _ => false - } -} - } private def classLoaderRef(c: Class[_]): HashableWeakReference = { -new HashableWeakReference(c.getClassLoader) +new HashableWeakReference(c.getClassLoader, queue) } private[this] val outerScopes = { @@ -154,3 +135,31 @@ object OuterScopes { // e.g. `ammonite.$sess.cmd8$Helper$Foo` -> `ammonite.$sess.cmd8.instance.Foo` private[this] val AmmoniteREPLClass = """^(ammonite\.\$sess\.cmd(?:\d+)\$).*""".r } + +/** + * A [[WeakReference]] that has a stable hash-key. When the referent is still alive we will use + * the referent for equality, once it is dead it we will fallback to referential equality. This + * way you can still do lookups in a map when the referent is alive, and are capable of removing + * dead entries after GC (using a [[ReferenceQueue]]). + */ +private[catalyst] class HashableWeakReference(v: AnyRef, queue: ReferenceQueue[AnyRef]) + extends WeakReference[AnyRef](v, queue) { + def this(v: AnyRef) = this(v, null) + private[this] val hash = v.hashCode() + override def hashCode(): Int = hash + override def equals(obj: Any): Boolean = { +obj match { + case other: HashableWeakReference => +// Note that referential equality is used to identify & purge +// references from the map whose' referent went out of scope. +if (this eq other) { + true +} else { + val referent = get() + val otherReferent = other.get() + referent != null && otherReferent != null && Objects.equals(referent, otherReferent) +} + case _ => false +} + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 59688cae889..fe61cc81359 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.expressions.codegen import java.io.ByteArrayInputStream -import java.util.UUID import scala.annotation.tailrec import scala.collection.JavaConverters._ @@ -26,7 +25,6 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal -import com.google.common.cache.{CacheBuilder, CacheLoader} import com.google.common.util.concurrent.{ExecutionError, UncheckedExecutionException} import org.codehaus.commons.compiler.{CompileException, InternalCompilerException} import org.codehaus.janino.ClassBodyEvaluator @@ -37,6 +35,7 @@ import org.apache.spark.executor.InputMetrics import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.CodegenMetrics import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.HashableWeakReference import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.cat
[spark] branch branch-3.5 updated: [SPARK-44791][CONNECT] Make ArrowDeserializer work with REPL generated classes
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new f0bb1391fe4 [SPARK-44791][CONNECT] Make ArrowDeserializer work with REPL generated classes f0bb1391fe4 is described below commit f0bb1391fe460fee886bce9151a47e89e75de671 Author: Herman van Hovell AuthorDate: Mon Aug 14 02:38:54 2023 +0200 [SPARK-44791][CONNECT] Make ArrowDeserializer work with REPL generated classes ### What changes were proposed in this pull request? Connects arrow deserialization currently does not work with REPL generated classes. For example the following code would fail: ```scala case class MyTestClass(value: Int) { override def toString: String = value.toString } spark.range(10).map(i => MyTestClass(i.toInt)).collect() ``` The problem is that for instantiation of the `MyTestClass` class we need the instance of the class that it was defined in (its outerscope). In Spark we have a mechanism called `OuterScopes` to register these instances in. The `ArrowDeserializer` was not resolving this outer instance. This PR fixes this. We have a similar issue on the executor/driver side. This will be fixed in a different PR. ### Why are the changes needed? It is a bug. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I have added tests to `ReplE2Esuite` and to the `ArrowEncoderSuite`. Closes #42473 from hvanhovell/SPARK-44791. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit dcf3d582293c3dbb3820d12fa15b41e8bd5fe6ad) Signed-off-by: Herman van Hovell --- .../org/apache/spark/util/SparkClassUtils.scala| 28 +++ .../connect/client/arrow/ArrowDeserializer.scala | 14 +++- .../spark/sql/application/ReplE2ESuite.scala | 33 - .../connect/client/arrow/ArrowEncoderSuite.scala | 12 ++- .../main/scala/org/apache/spark/util/Utils.scala | 28 --- .../spark/sql/catalyst/encoders/OuterScopes.scala | 85 +- .../apache/spark/sql/errors/ExecutionErrors.scala | 7 ++ .../spark/sql/errors/QueryExecutionErrors.scala| 7 -- 8 files changed, 138 insertions(+), 76 deletions(-) diff --git a/common/utils/src/main/scala/org/apache/spark/util/SparkClassUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/SparkClassUtils.scala index a237869aef3..679d546d04c 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/SparkClassUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/SparkClassUtils.scala @@ -50,6 +50,34 @@ trait SparkClassUtils { def classIsLoadable(clazz: String): Boolean = { Try { classForName(clazz, initialize = false) }.isSuccess } + + /** + * Returns true if and only if the underlying class is a member class. + * + * Note: jdk8u throws a "Malformed class name" error if a given class is a deeply-nested + * inner class (See SPARK-34607 for details). This issue has already been fixed in jdk9+, so + * we can remove this helper method safely if we drop the support of jdk8u. + */ + def isMemberClass(cls: Class[_]): Boolean = { +try { + cls.isMemberClass +} catch { + case _: InternalError => +// We emulate jdk8u `Class.isMemberClass` below: +// public boolean isMemberClass() { +// return getSimpleBinaryName() != null && !isLocalOrAnonymousClass(); +// } +// `getSimpleBinaryName()` returns null if a given class is a top-level class, +// so we replace it with `cls.getEnclosingClass != null`. The second condition checks +// if a given class is not a local or an anonymous class, so we replace it with +// `cls.getEnclosingMethod == null` because `cls.getEnclosingMethod()` return a value +// only in either case (JVM Spec 4.8.6). +// +// Note: The newer jdk evaluates `!isLocalOrAnonymousClass()` first, +// we reorder the conditions to follow it. +cls.getEnclosingMethod == null && cls.getEnclosingClass != null +} + } } object SparkClassUtils extends SparkClassUtils diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala index 55dd640f1b6..82086b9d47a 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala @@ -34,7 +34,7 @@ import org.apache.arrow.vector.ipc.ArrowReader import org.apache.
[spark] branch master updated: [SPARK-44791][CONNECT] Make ArrowDeserializer work with REPL generated classes
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new dcf3d582293 [SPARK-44791][CONNECT] Make ArrowDeserializer work with REPL generated classes dcf3d582293 is described below commit dcf3d582293c3dbb3820d12fa15b41e8bd5fe6ad Author: Herman van Hovell AuthorDate: Mon Aug 14 02:38:54 2023 +0200 [SPARK-44791][CONNECT] Make ArrowDeserializer work with REPL generated classes ### What changes were proposed in this pull request? Connects arrow deserialization currently does not work with REPL generated classes. For example the following code would fail: ```scala case class MyTestClass(value: Int) { override def toString: String = value.toString } spark.range(10).map(i => MyTestClass(i.toInt)).collect() ``` The problem is that for instantiation of the `MyTestClass` class we need the instance of the class that it was defined in (its outerscope). In Spark we have a mechanism called `OuterScopes` to register these instances in. The `ArrowDeserializer` was not resolving this outer instance. This PR fixes this. We have a similar issue on the executor/driver side. This will be fixed in a different PR. ### Why are the changes needed? It is a bug. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I have added tests to `ReplE2Esuite` and to the `ArrowEncoderSuite`. Closes #42473 from hvanhovell/SPARK-44791. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../org/apache/spark/util/SparkClassUtils.scala| 28 +++ .../connect/client/arrow/ArrowDeserializer.scala | 14 +++- .../spark/sql/application/ReplE2ESuite.scala | 33 - .../connect/client/arrow/ArrowEncoderSuite.scala | 12 ++- .../main/scala/org/apache/spark/util/Utils.scala | 28 --- .../spark/sql/catalyst/encoders/OuterScopes.scala | 85 +- .../apache/spark/sql/errors/ExecutionErrors.scala | 7 ++ .../spark/sql/errors/QueryExecutionErrors.scala| 7 -- 8 files changed, 138 insertions(+), 76 deletions(-) diff --git a/common/utils/src/main/scala/org/apache/spark/util/SparkClassUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/SparkClassUtils.scala index a237869aef3..679d546d04c 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/SparkClassUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/SparkClassUtils.scala @@ -50,6 +50,34 @@ trait SparkClassUtils { def classIsLoadable(clazz: String): Boolean = { Try { classForName(clazz, initialize = false) }.isSuccess } + + /** + * Returns true if and only if the underlying class is a member class. + * + * Note: jdk8u throws a "Malformed class name" error if a given class is a deeply-nested + * inner class (See SPARK-34607 for details). This issue has already been fixed in jdk9+, so + * we can remove this helper method safely if we drop the support of jdk8u. + */ + def isMemberClass(cls: Class[_]): Boolean = { +try { + cls.isMemberClass +} catch { + case _: InternalError => +// We emulate jdk8u `Class.isMemberClass` below: +// public boolean isMemberClass() { +// return getSimpleBinaryName() != null && !isLocalOrAnonymousClass(); +// } +// `getSimpleBinaryName()` returns null if a given class is a top-level class, +// so we replace it with `cls.getEnclosingClass != null`. The second condition checks +// if a given class is not a local or an anonymous class, so we replace it with +// `cls.getEnclosingMethod == null` because `cls.getEnclosingMethod()` return a value +// only in either case (JVM Spec 4.8.6). +// +// Note: The newer jdk evaluates `!isLocalOrAnonymousClass()` first, +// we reorder the conditions to follow it. +cls.getEnclosingMethod == null && cls.getEnclosingClass != null +} + } } object SparkClassUtils extends SparkClassUtils diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala index 55dd640f1b6..82086b9d47a 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala @@ -34,7 +34,7 @@ import org.apache.arrow.vector.ipc.ArrowReader import org.apache.arrow.vector.util.Text import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.
[spark] branch branch-3.5 updated: [SPARK-44736][CONNECT] Add Dataset.explode to Spark Connect Scala Client
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new b0b15475a0a [SPARK-44736][CONNECT] Add Dataset.explode to Spark Connect Scala Client b0b15475a0a is described below commit b0b15475a0ac2d73b829491532747a249498c1a6 Author: Herman van Hovell AuthorDate: Sun Aug 13 20:27:08 2023 +0200 [SPARK-44736][CONNECT] Add Dataset.explode to Spark Connect Scala Client ### What changes were proposed in this pull request? This PR adds Dataset.explode to the Spark Connect Scala Client. ### Why are the changes needed? To increase compatibility with the existing Dataset API in sql/core. ### Does this PR introduce _any_ user-facing change? Yes, it adds a new method to the scala client. ### How was this patch tested? I added a test to `UserDefinedFunctionE2ETestSuite`. Closes #42418 from hvanhovell/SPARK-44736. Lead-authored-by: Herman van Hovell Co-authored-by: itholic Co-authored-by: Juliusz Sompolski Co-authored-by: Martin Grund Co-authored-by: Hyukjin Kwon Co-authored-by: Kent Yao Co-authored-by: Wenchen Fan Co-authored-by: Wei Liu Co-authored-by: Ruifeng Zheng Co-authored-by: Gengliang Wang Co-authored-by: Yuming Wang Co-authored-by: Herman van Hovell Co-authored-by: 余良 Co-authored-by: Dongjoon Hyun Co-authored-by: Jack Chen Co-authored-by: srielau Co-authored-by: zhyhimont Co-authored-by: Daniel Tenedorio Co-authored-by: Dongjoon Hyun Co-authored-by: Zhyhimont Dmitry Co-authored-by: Sandip Agarwala <131817656+sandip...@users.noreply.github.com> Co-authored-by: yangjie01 Co-authored-by: Yihong He Co-authored-by: Rameshkrishnan Muthusamy Co-authored-by: Jia Fan Co-authored-by: allisonwang-db Co-authored-by: Utkarsh Co-authored-by: Cheng Pan Co-authored-by: Jason Li Co-authored-by: Shu Wang Co-authored-by: Nicolas Fraison Co-authored-by: Max Gekk Co-authored-by: panbingkun Co-authored-by: Ziqi Liu Signed-off-by: Herman van Hovell (cherry picked from commit f496cd1ee2a7e59af08e1bd3ab0579f93cc46da9) Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 70 ++ .../sql/UserDefinedFunctionE2ETestSuite.scala | 60 +++ .../CheckConnectJvmClientCompatibility.scala | 1 - .../apache/spark/sql/connect/common/UdfUtils.scala | 4 ++ .../sql/connect/planner/SparkConnectPlanner.scala | 3 +- 5 files changed, 136 insertions(+), 2 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 2d72ea6bda8..28b04fb850e 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -21,12 +21,14 @@ import java.util.{Collections, Locale} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.function._ import org.apache.spark.connect.proto +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ import org.apache.spark.sql.catalyst.expressions.OrderUtils @@ -2728,6 +2730,74 @@ class Dataset[T] private[sql] ( flatMap(UdfUtils.flatMapFuncToScalaFunc(f))(encoder) } + /** + * (Scala-specific) Returns a new Dataset where each row has been expanded to zero or more rows + * by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. The columns of the + * input row are implicitly joined with each row that is output by the function. + * + * Given that this is deprecated, as an alternative, you can explode columns either using + * `functions.explode()` or `flatMap()`. The following example uses these alternatives to count + * the number of books that contain a given word: + * + * {{{ + * case class Book(title: String, words: String) + * val ds: Dataset[Book] + * + * val allWords = ds.select($"title", explode(split($"words", " ")).as("word")) + * + * val bookCountPerWord = allWords.groupBy("word").agg(count_distinct("title")) + * }}} + * + * Using `flatMap()` this can similarly be exploded as: + * + * {{{ + * ds.flatMap(_.words.spli
[spark] branch master updated: [SPARK-44736][CONNECT] Add Dataset.explode to Spark Connect Scala Client
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f496cd1ee2a [SPARK-44736][CONNECT] Add Dataset.explode to Spark Connect Scala Client f496cd1ee2a is described below commit f496cd1ee2a7e59af08e1bd3ab0579f93cc46da9 Author: Herman van Hovell AuthorDate: Sun Aug 13 20:27:08 2023 +0200 [SPARK-44736][CONNECT] Add Dataset.explode to Spark Connect Scala Client ### What changes were proposed in this pull request? This PR adds Dataset.explode to the Spark Connect Scala Client. ### Why are the changes needed? To increase compatibility with the existing Dataset API in sql/core. ### Does this PR introduce _any_ user-facing change? Yes, it adds a new method to the scala client. ### How was this patch tested? I added a test to `UserDefinedFunctionE2ETestSuite`. Closes #42418 from hvanhovell/SPARK-44736. Lead-authored-by: Herman van Hovell Co-authored-by: itholic Co-authored-by: Juliusz Sompolski Co-authored-by: Martin Grund Co-authored-by: Hyukjin Kwon Co-authored-by: Kent Yao Co-authored-by: Wenchen Fan Co-authored-by: Wei Liu Co-authored-by: Ruifeng Zheng Co-authored-by: Gengliang Wang Co-authored-by: Yuming Wang Co-authored-by: Herman van Hovell Co-authored-by: 余良 Co-authored-by: Dongjoon Hyun Co-authored-by: Jack Chen Co-authored-by: srielau Co-authored-by: zhyhimont Co-authored-by: Daniel Tenedorio Co-authored-by: Dongjoon Hyun Co-authored-by: Zhyhimont Dmitry Co-authored-by: Sandip Agarwala <131817656+sandip...@users.noreply.github.com> Co-authored-by: yangjie01 Co-authored-by: Yihong He Co-authored-by: Rameshkrishnan Muthusamy Co-authored-by: Jia Fan Co-authored-by: allisonwang-db Co-authored-by: Utkarsh Co-authored-by: Cheng Pan Co-authored-by: Jason Li Co-authored-by: Shu Wang Co-authored-by: Nicolas Fraison Co-authored-by: Max Gekk Co-authored-by: panbingkun Co-authored-by: Ziqi Liu Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 70 ++ .../sql/UserDefinedFunctionE2ETestSuite.scala | 60 +++ .../CheckConnectJvmClientCompatibility.scala | 1 - .../apache/spark/sql/connect/common/UdfUtils.scala | 4 ++ .../sql/connect/planner/SparkConnectPlanner.scala | 3 +- 5 files changed, 136 insertions(+), 2 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 2d72ea6bda8..28b04fb850e 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -21,12 +21,14 @@ import java.util.{Collections, Locale} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.function._ import org.apache.spark.connect.proto +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ import org.apache.spark.sql.catalyst.expressions.OrderUtils @@ -2728,6 +2730,74 @@ class Dataset[T] private[sql] ( flatMap(UdfUtils.flatMapFuncToScalaFunc(f))(encoder) } + /** + * (Scala-specific) Returns a new Dataset where each row has been expanded to zero or more rows + * by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. The columns of the + * input row are implicitly joined with each row that is output by the function. + * + * Given that this is deprecated, as an alternative, you can explode columns either using + * `functions.explode()` or `flatMap()`. The following example uses these alternatives to count + * the number of books that contain a given word: + * + * {{{ + * case class Book(title: String, words: String) + * val ds: Dataset[Book] + * + * val allWords = ds.select($"title", explode(split($"words", " ")).as("word")) + * + * val bookCountPerWord = allWords.groupBy("word").agg(count_distinct("title")) + * }}} + * + * Using `flatMap()` this can similarly be exploded as: + * + * {{{ + * ds.flatMap(_.words.split(" ")) + * }}} + * + * @group untypedrel + * @since 3.5.0 + */ + @deprecated("use fl
[spark] branch branch-3.5 updated: [SPARK-44625][CONNECT] SparkConnectExecutionManager to track all executions
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 74f6abef94c [SPARK-44625][CONNECT] SparkConnectExecutionManager to track all executions 74f6abef94c is described below commit 74f6abef94c4cfb12fe36f8050138780bd669652 Author: Juliusz Sompolski AuthorDate: Fri Aug 11 18:32:07 2023 +0200 [SPARK-44625][CONNECT] SparkConnectExecutionManager to track all executions ### What changes were proposed in this pull request? SparkConnectExecutionManager tracks all executions (ExecuteHolder) in all sessions of Spark Connect. It tracks which executions have RPCs (ExecuteGrpcReponseSender) attached to them. If an execution gets abandoned (it's not cleared with ReleaseExecute by the client, but no new RPC arrives), it will be automatically interrupted and removed after a timeout. Note for the failure: ``` Error: Field "2" on message "ReleaseExecuteResponse" moved from outside to inside a oneof. Error: buf found 1 breaking changes. ``` The message ReleaseExecuteResponse has not been released yet, so it's not a breaking change compared to any released version. ### Why are the changes needed? Need the SparkConnectExecutionManager to track reattachable executions that got abandoned by client. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pending, but the SparkConnectExecutionManager allows tests to inspect execution state, so allows writing more unit tests about reattachable execution. Closes #42423 from juliuszsompolski/SPARK-44625. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell (cherry picked from commit 84dbe846b30d5250169b834b182779a104570888) Signed-off-by: Herman van Hovell --- .../src/main/resources/error/error-classes.json| 5 + .../src/main/protobuf/spark/connect/base.proto | 8 +- .../apache/spark/sql/connect/config/Connect.scala | 24 +++ .../execution/ExecuteGrpcResponseSender.scala | 119 ++-- .../execution/ExecuteResponseObserver.scala| 9 +- .../spark/sql/connect/service/ExecuteHolder.scala | 143 +++--- .../spark/sql/connect/service/SessionHolder.scala | 39 ++-- .../service/SparkConnectExecutePlanHandler.scala | 19 +- .../service/SparkConnectExecutionManager.scala | 209 + .../SparkConnectReattachExecuteHandler.scala | 31 ++- .../SparkConnectReleaseExecuteHandler.scala| 51 ++--- .../sql/connect/service/SparkConnectService.scala | 11 ++ .../spark/sql/connect/utils/ErrorUtils.scala | 15 +- .../connect/planner/SparkConnectPlannerSuite.scala | 5 +- ...-error-conditions-invalid-handle-error-class.md | 4 + python/pyspark/sql/connect/proto/base_pb2.py | 8 +- python/pyspark/sql/connect/proto/base_pb2.pyi | 23 ++- python/pyspark/sql/connect/proto/base_pb2_grpc.py | 2 +- 18 files changed, 566 insertions(+), 159 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index d9d1963c958..74542f2b914 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1426,6 +1426,11 @@ "Handle must be an UUID string of the format '00112233-4455-6677-8899-aabbccddeeff'" ] }, + "OPERATION_ABANDONED" : { +"message" : [ + "Operation was considered abandoned because of inactivity and removed." +] + }, "OPERATION_ALREADY_EXISTS" : { "message" : [ "Operation already exists." diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 79dbadba5bb..65e2493f836 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -772,8 +772,10 @@ message ReleaseExecuteResponse { // Session id in which the release was running. string session_id = 1; - // Operation id of the operation which the release concerns. - string operation_id = 2; + // Operation id of the operation on which the release executed. + // If the operation couldn't be found (because e.g. it was concurrently released), will be unset. + // Otherwise, it will be equal to the operation_id from request. + optional string operation_id = 2; } // Main interface for the SparkConnect service. @@ -809,7 +811,7 @@ service SparkConnectService { // Release an reattachabl
[spark] branch master updated: [SPARK-44625][CONNECT] SparkConnectExecutionManager to track all executions
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 84dbe846b30 [SPARK-44625][CONNECT] SparkConnectExecutionManager to track all executions 84dbe846b30 is described below commit 84dbe846b30d5250169b834b182779a104570888 Author: Juliusz Sompolski AuthorDate: Fri Aug 11 18:32:07 2023 +0200 [SPARK-44625][CONNECT] SparkConnectExecutionManager to track all executions ### What changes were proposed in this pull request? SparkConnectExecutionManager tracks all executions (ExecuteHolder) in all sessions of Spark Connect. It tracks which executions have RPCs (ExecuteGrpcReponseSender) attached to them. If an execution gets abandoned (it's not cleared with ReleaseExecute by the client, but no new RPC arrives), it will be automatically interrupted and removed after a timeout. Note for the failure: ``` Error: Field "2" on message "ReleaseExecuteResponse" moved from outside to inside a oneof. Error: buf found 1 breaking changes. ``` The message ReleaseExecuteResponse has not been released yet, so it's not a breaking change compared to any released version. ### Why are the changes needed? Need the SparkConnectExecutionManager to track reattachable executions that got abandoned by client. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pending, but the SparkConnectExecutionManager allows tests to inspect execution state, so allows writing more unit tests about reattachable execution. Closes #42423 from juliuszsompolski/SPARK-44625. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell --- .../src/main/resources/error/error-classes.json| 5 + .../src/main/protobuf/spark/connect/base.proto | 8 +- .../apache/spark/sql/connect/config/Connect.scala | 24 +++ .../execution/ExecuteGrpcResponseSender.scala | 119 ++-- .../execution/ExecuteResponseObserver.scala| 9 +- .../spark/sql/connect/service/ExecuteHolder.scala | 143 +++--- .../spark/sql/connect/service/SessionHolder.scala | 39 ++-- .../service/SparkConnectExecutePlanHandler.scala | 19 +- .../service/SparkConnectExecutionManager.scala | 209 + .../SparkConnectReattachExecuteHandler.scala | 31 ++- .../SparkConnectReleaseExecuteHandler.scala| 51 ++--- .../sql/connect/service/SparkConnectService.scala | 11 ++ .../spark/sql/connect/utils/ErrorUtils.scala | 15 +- .../connect/planner/SparkConnectPlannerSuite.scala | 5 +- ...-error-conditions-invalid-handle-error-class.md | 4 + python/pyspark/sql/connect/proto/base_pb2.py | 8 +- python/pyspark/sql/connect/proto/base_pb2.pyi | 23 ++- python/pyspark/sql/connect/proto/base_pb2_grpc.py | 2 +- 18 files changed, 566 insertions(+), 159 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 75125d2275d..133c2dd826c 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1552,6 +1552,11 @@ "Handle must be an UUID string of the format '00112233-4455-6677-8899-aabbccddeeff'" ] }, + "OPERATION_ABANDONED" : { +"message" : [ + "Operation was considered abandoned because of inactivity and removed." +] + }, "OPERATION_ALREADY_EXISTS" : { "message" : [ "Operation already exists." diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 79dbadba5bb..65e2493f836 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -772,8 +772,10 @@ message ReleaseExecuteResponse { // Session id in which the release was running. string session_id = 1; - // Operation id of the operation which the release concerns. - string operation_id = 2; + // Operation id of the operation on which the release executed. + // If the operation couldn't be found (because e.g. it was concurrently released), will be unset. + // Otherwise, it will be equal to the operation_id from request. + optional string operation_id = 2; } // Main interface for the SparkConnect service. @@ -809,7 +811,7 @@ service SparkConnectService { // Release an reattachable execution, or parts thereof. // The ExecutePlan must have been started with ReattachOptions.reattachable=true
[spark] branch master updated: [SPARK-44761][CONNECT] Support DataStreamWriter.foreachBatch(VoidFunction2)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5a27dc9b617 [SPARK-44761][CONNECT] Support DataStreamWriter.foreachBatch(VoidFunction2) 5a27dc9b617 is described below commit 5a27dc9b6174fedefd08bcfe8a5b42bdfde2b7f6 Author: Herman van Hovell AuthorDate: Fri Aug 11 14:36:22 2023 +0200 [SPARK-44761][CONNECT] Support DataStreamWriter.foreachBatch(VoidFunction2) ### What changes were proposed in this pull request? This PR adds the `DatastreamWriter.foreachBatch(VoidFunction2). ### Why are the changes needed? To increase binary compatibility with the APIs in `sql/core`. ### Does this PR introduce _any_ user-facing change? Yes. It adds a new method to DatastreamWriter. ### How was this patch tested? I modified an existing code path. Closes #42430 from hvanhovell/SPARK-44761. Lead-authored-by: Herman van Hovell Co-authored-by: Hyukjin Kwon Signed-off-by: Herman van Hovell --- .../spark/sql/streaming/DataStreamWriter.scala | 22 -- .../CheckConnectJvmClientCompatibility.scala | 3 --- .../sql/streaming/ClientStreamingQuerySuite.scala | 9 ++--- .../apache/spark/sql/connect/common/UdfUtils.scala | 3 +++ 4 files changed, 29 insertions(+), 8 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index b9aa1f5bc58..54eb6e76140 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -25,13 +25,13 @@ import scala.collection.JavaConverters._ import com.google.protobuf.ByteString import org.apache.spark.annotation.Evolving +import org.apache.spark.api.java.function.VoidFunction2 import org.apache.spark.connect.proto import org.apache.spark.connect.proto.Command import org.apache.spark.connect.proto.WriteStreamOperationStart import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, ForeachWriter} -import org.apache.spark.sql.connect.common.DataTypeProtoConverter -import org.apache.spark.sql.connect.common.ForeachWriterPacket +import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, UdfUtils} import org.apache.spark.sql.execution.streaming.AvailableNowTrigger import org.apache.spark.sql.execution.streaming.ContinuousTrigger import org.apache.spark.sql.execution.streaming.OneTimeTrigger @@ -247,6 +247,24 @@ final class DataStreamWriter[T] private[sql] (ds: Dataset[T]) extends Logging { this } + /** + * :: Experimental :: + * + * (Java-specific) Sets the output of the streaming query to be processed using the provided + * function. This is supported only in the micro-batch execution modes (that is, when the + * trigger is not continuous). In every micro-batch, the provided function will be called in + * every micro-batch with (i) the output rows as a Dataset and (ii) the batch identifier. The + * batchId can be used to deduplicate and transactionally write the output (that is, the + * provided Dataset) to external systems. The output Dataset is guaranteed to be exactly the + * same for the same batchId (assuming all operations are deterministic in the query). + * + * @since 3.5.0 + */ + @Evolving + def foreachBatch(function: VoidFunction2[Dataset[T], java.lang.Long]): DataStreamWriter[T] = { +foreachBatch(UdfUtils.foreachBatchFuncToScalaFunc(function)) + } + /** * Starts the execution of the streaming query, which will continually output results to the * given path as new data arrives. The returned [[StreamingQuery]] object can be used to diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index 3fc02d7c397..04b162eceec 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -234,9 +234,6 @@ object CheckConnectJvmClientCompatibility { // DataStreamWriter ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.streaming.DataStreamWriter$"), - ProblemFilters.exclude[Problem]( -"org.apache.spark.sql.streaming.DataStreamWriter.foreachBatch"
[spark] branch branch-3.5 updated: [SPARK-44761][CONNECT] Support DataStreamWriter.foreachBatch(VoidFunction2)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new b11022f042f [SPARK-44761][CONNECT] Support DataStreamWriter.foreachBatch(VoidFunction2) b11022f042f is described below commit b11022f042f3264e24e3c202315e3eb2b8be1b14 Author: Herman van Hovell AuthorDate: Fri Aug 11 14:36:22 2023 +0200 [SPARK-44761][CONNECT] Support DataStreamWriter.foreachBatch(VoidFunction2) ### What changes were proposed in this pull request? This PR adds the `DatastreamWriter.foreachBatch(VoidFunction2). ### Why are the changes needed? To increase binary compatibility with the APIs in `sql/core`. ### Does this PR introduce _any_ user-facing change? Yes. It adds a new method to DatastreamWriter. ### How was this patch tested? I modified an existing code path. Closes #42430 from hvanhovell/SPARK-44761. Lead-authored-by: Herman van Hovell Co-authored-by: Hyukjin Kwon Signed-off-by: Herman van Hovell (cherry picked from commit 5a27dc9b6174fedefd08bcfe8a5b42bdfde2b7f6) Signed-off-by: Herman van Hovell --- .../spark/sql/streaming/DataStreamWriter.scala | 22 -- .../CheckConnectJvmClientCompatibility.scala | 3 --- .../sql/streaming/ClientStreamingQuerySuite.scala | 9 ++--- .../apache/spark/sql/connect/common/UdfUtils.scala | 3 +++ 4 files changed, 29 insertions(+), 8 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index b9aa1f5bc58..54eb6e76140 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -25,13 +25,13 @@ import scala.collection.JavaConverters._ import com.google.protobuf.ByteString import org.apache.spark.annotation.Evolving +import org.apache.spark.api.java.function.VoidFunction2 import org.apache.spark.connect.proto import org.apache.spark.connect.proto.Command import org.apache.spark.connect.proto.WriteStreamOperationStart import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, ForeachWriter} -import org.apache.spark.sql.connect.common.DataTypeProtoConverter -import org.apache.spark.sql.connect.common.ForeachWriterPacket +import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, UdfUtils} import org.apache.spark.sql.execution.streaming.AvailableNowTrigger import org.apache.spark.sql.execution.streaming.ContinuousTrigger import org.apache.spark.sql.execution.streaming.OneTimeTrigger @@ -247,6 +247,24 @@ final class DataStreamWriter[T] private[sql] (ds: Dataset[T]) extends Logging { this } + /** + * :: Experimental :: + * + * (Java-specific) Sets the output of the streaming query to be processed using the provided + * function. This is supported only in the micro-batch execution modes (that is, when the + * trigger is not continuous). In every micro-batch, the provided function will be called in + * every micro-batch with (i) the output rows as a Dataset and (ii) the batch identifier. The + * batchId can be used to deduplicate and transactionally write the output (that is, the + * provided Dataset) to external systems. The output Dataset is guaranteed to be exactly the + * same for the same batchId (assuming all operations are deterministic in the query). + * + * @since 3.5.0 + */ + @Evolving + def foreachBatch(function: VoidFunction2[Dataset[T], java.lang.Long]): DataStreamWriter[T] = { +foreachBatch(UdfUtils.foreachBatchFuncToScalaFunc(function)) + } + /** * Starts the execution of the streaming query, which will continually output results to the * given path as new data arrives. The returned [[StreamingQuery]] object can be used to diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index 3fc02d7c397..04b162eceec 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -234,9 +234,6 @@ object CheckConnectJvmClientCompatibility { // DataStreamWriter ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.streaming.DataStreamWriter$"), - ProblemFilters.exclu
[spark] branch branch-3.5 updated: [SPARK-44720][CONNECT] Make Dataset use Encoder instead of AgnosticEncoder
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 6eca5da8d3f [SPARK-44720][CONNECT] Make Dataset use Encoder instead of AgnosticEncoder 6eca5da8d3f is described below commit 6eca5da8d3fba6d1e385f06494030996241937fa Author: Herman van Hovell AuthorDate: Wed Aug 9 15:58:18 2023 +0200 [SPARK-44720][CONNECT] Make Dataset use Encoder instead of AgnosticEncoder ### What changes were proposed in this pull request? Make the Spark Connect Dataset use Encoder instead of AgnosticEncoder ### Why are the changes needed? We want to improve binary compatibility between the Spark Connect Scala Client and the original sql/core APIs. ### Does this PR introduce _any_ user-facing change? Yes. It changes the type of `Dataset.encoder` from `AgnosticEncoder` to `Encoder`. ### How was this patch tested? Existing tests. Closes #42396 from hvanhovell/SPARK-44720. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit be9ffb37585fe421705ceaa52fe49b89c50703a3) Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 87 -- .../apache/spark/sql/KeyValueGroupedDataset.scala | 6 +- .../spark/sql/streaming/DataStreamWriter.scala | 2 +- .../CheckConnectJvmClientCompatibility.scala | 3 - 4 files changed, 50 insertions(+), 48 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 5f263903c8b..2d72ea6bda8 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -128,11 +128,13 @@ import org.apache.spark.util.SparkClassUtils class Dataset[T] private[sql] ( val sparkSession: SparkSession, @DeveloperApi val plan: proto.Plan, -val encoder: AgnosticEncoder[T]) +val encoder: Encoder[T]) extends Serializable { // Make sure we don't forget to set plan id. assert(plan.getRoot.getCommon.hasPlanId) + private[sql] val agnosticEncoder: AgnosticEncoder[T] = encoderFor(encoder) + override def toString: String = { try { val builder = new mutable.StringBuilder @@ -828,7 +830,7 @@ class Dataset[T] private[sql] ( } private def buildSort(global: Boolean, sortExprs: Seq[Column]): Dataset[T] = { -sparkSession.newDataset(encoder) { builder => +sparkSession.newDataset(agnosticEncoder) { builder => builder.getSortBuilder .setInput(plan.getRoot) .setIsGlobal(global) @@ -878,8 +880,8 @@ class Dataset[T] private[sql] ( ProductEncoder[(T, U)]( ClassTag(SparkClassUtils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple2")), Seq( - EncoderField(s"_1", this.encoder, leftNullable, Metadata.empty), - EncoderField(s"_2", other.encoder, rightNullable, Metadata.empty))) + EncoderField(s"_1", this.agnosticEncoder, leftNullable, Metadata.empty), + EncoderField(s"_2", other.agnosticEncoder, rightNullable, Metadata.empty))) sparkSession.newDataset(tupleEncoder) { builder => val joinBuilder = builder.getJoinBuilder @@ -889,8 +891,8 @@ class Dataset[T] private[sql] ( .setJoinType(joinTypeValue) .setJoinCondition(condition.expr) .setJoinDataType(joinBuilder.getJoinDataTypeBuilder - .setIsLeftStruct(this.encoder.isStruct) - .setIsRightStruct(other.encoder.isStruct)) + .setIsLeftStruct(this.agnosticEncoder.isStruct) + .setIsRightStruct(other.agnosticEncoder.isStruct)) } } @@ -1010,13 +1012,13 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ @scala.annotation.varargs - def hint(name: String, parameters: Any*): Dataset[T] = sparkSession.newDataset(encoder) { -builder => + def hint(name: String, parameters: Any*): Dataset[T] = +sparkSession.newDataset(agnosticEncoder) { builder => builder.getHintBuilder .setInput(plan.getRoot) .setName(name) .addAllParameters(parameters.map(p => functions.lit(p).expr).asJava) - } +} private def getPlanId: Option[Long] = if (plan.getRoot.hasCommon && plan.getRoot.getCommon.hasPlanId) { @@ -1056,7 +1058,7 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def as(alias: String): Dataset[T] = sparkSession.newDataset(encoder) { builder => + def as(alias: String): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => builder.getSubquer
[spark] branch master updated: [SPARK-44720][CONNECT] Make Dataset use Encoder instead of AgnosticEncoder
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new be9ffb37585 [SPARK-44720][CONNECT] Make Dataset use Encoder instead of AgnosticEncoder be9ffb37585 is described below commit be9ffb37585fe421705ceaa52fe49b89c50703a3 Author: Herman van Hovell AuthorDate: Wed Aug 9 15:58:18 2023 +0200 [SPARK-44720][CONNECT] Make Dataset use Encoder instead of AgnosticEncoder ### What changes were proposed in this pull request? Make the Spark Connect Dataset use Encoder instead of AgnosticEncoder ### Why are the changes needed? We want to improve binary compatibility between the Spark Connect Scala Client and the original sql/core APIs. ### Does this PR introduce _any_ user-facing change? Yes. It changes the type of `Dataset.encoder` from `AgnosticEncoder` to `Encoder`. ### How was this patch tested? Existing tests. Closes #42396 from hvanhovell/SPARK-44720. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 87 -- .../apache/spark/sql/KeyValueGroupedDataset.scala | 6 +- .../spark/sql/streaming/DataStreamWriter.scala | 2 +- .../CheckConnectJvmClientCompatibility.scala | 3 - 4 files changed, 50 insertions(+), 48 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 5f263903c8b..2d72ea6bda8 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -128,11 +128,13 @@ import org.apache.spark.util.SparkClassUtils class Dataset[T] private[sql] ( val sparkSession: SparkSession, @DeveloperApi val plan: proto.Plan, -val encoder: AgnosticEncoder[T]) +val encoder: Encoder[T]) extends Serializable { // Make sure we don't forget to set plan id. assert(plan.getRoot.getCommon.hasPlanId) + private[sql] val agnosticEncoder: AgnosticEncoder[T] = encoderFor(encoder) + override def toString: String = { try { val builder = new mutable.StringBuilder @@ -828,7 +830,7 @@ class Dataset[T] private[sql] ( } private def buildSort(global: Boolean, sortExprs: Seq[Column]): Dataset[T] = { -sparkSession.newDataset(encoder) { builder => +sparkSession.newDataset(agnosticEncoder) { builder => builder.getSortBuilder .setInput(plan.getRoot) .setIsGlobal(global) @@ -878,8 +880,8 @@ class Dataset[T] private[sql] ( ProductEncoder[(T, U)]( ClassTag(SparkClassUtils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple2")), Seq( - EncoderField(s"_1", this.encoder, leftNullable, Metadata.empty), - EncoderField(s"_2", other.encoder, rightNullable, Metadata.empty))) + EncoderField(s"_1", this.agnosticEncoder, leftNullable, Metadata.empty), + EncoderField(s"_2", other.agnosticEncoder, rightNullable, Metadata.empty))) sparkSession.newDataset(tupleEncoder) { builder => val joinBuilder = builder.getJoinBuilder @@ -889,8 +891,8 @@ class Dataset[T] private[sql] ( .setJoinType(joinTypeValue) .setJoinCondition(condition.expr) .setJoinDataType(joinBuilder.getJoinDataTypeBuilder - .setIsLeftStruct(this.encoder.isStruct) - .setIsRightStruct(other.encoder.isStruct)) + .setIsLeftStruct(this.agnosticEncoder.isStruct) + .setIsRightStruct(other.agnosticEncoder.isStruct)) } } @@ -1010,13 +1012,13 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ @scala.annotation.varargs - def hint(name: String, parameters: Any*): Dataset[T] = sparkSession.newDataset(encoder) { -builder => + def hint(name: String, parameters: Any*): Dataset[T] = +sparkSession.newDataset(agnosticEncoder) { builder => builder.getHintBuilder .setInput(plan.getRoot) .setName(name) .addAllParameters(parameters.map(p => functions.lit(p).expr).asJava) - } +} private def getPlanId: Option[Long] = if (plan.getRoot.hasCommon && plan.getRoot.getCommon.hasPlanId) { @@ -1056,7 +1058,7 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def as(alias: String): Dataset[T] = sparkSession.newDataset(encoder) { builder => + def as(alias: String): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => builder.getSubqueryAliasBuilder .setInput(plan.getRoot) .setAlias(alias) @@ -1238,8 +1240,9 @@ class Dataset[T] private[sql] (
[spark] branch branch-3.5 updated: [SPARK-44709][CONNECT] Run ExecuteGrpcResponseSender in reattachable execute in new thread to fix flow control
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 5514f1a [SPARK-44709][CONNECT] Run ExecuteGrpcResponseSender in reattachable execute in new thread to fix flow control 5514f1a is described below commit 5514f1ac1b4bc181a3ea9a4c48d013c2d179 Author: Juliusz Sompolski AuthorDate: Tue Aug 8 18:32:25 2023 +0200 [SPARK-44709][CONNECT] Run ExecuteGrpcResponseSender in reattachable execute in new thread to fix flow control ### What changes were proposed in this pull request? If executePlan / reattachExecute handling is done directly on the GRPC thread, flow control OnReady events are getting queued until after the handler returns, so OnReadyHandler never gets notified until after the handler exits. The correct way to use it is for the handler to delegate work to another thread and exit. See https://github.com/grpc/grpc-java/issues/7361 Tidied up and added a lot of logging and statistics to ExecuteGrpcResponseSender and ExecuteResponseObserver to be able to observe this behaviour. Followup work in https://issues.apache.org/jira/browse/SPARK-44625 is needed for cleanup of abandoned executions that will also make sure that these threads are joined. ### Why are the changes needed? ExecuteGrpcResponseSender gets stuck waiting on grpcCallObserverReadySignal because events from OnReadyHandler do not arrive. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added extensive debugging to ExecuteGrpcResponseSender and ExecuteResponseObserver and tested and observer the behaviour of all the threads. Closes #42355 from juliuszsompolski/spark-rpc-extra-thread. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell (cherry picked from commit 418bba5ad6053449a141f3c9c31ed3ad998995b8) Signed-off-by: Herman van Hovell --- .../apache/spark/sql/connect/config/Connect.scala | 13 +- .../connect/execution/CachedStreamResponse.scala | 2 + .../execution/ExecuteGrpcResponseSender.scala | 164 +++-- .../execution/ExecuteResponseObserver.scala| 116 --- .../connect/execution/ExecuteThreadRunner.scala| 3 +- .../spark/sql/connect/service/ExecuteHolder.scala | 21 ++- .../service/SparkConnectExecutePlanHandler.scala | 20 +-- .../SparkConnectReattachExecuteHandler.scala | 22 +-- 8 files changed, 264 insertions(+), 97 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala index 767b98983c4..51a9a1bf951 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala @@ -68,6 +68,17 @@ object Connect { .intConf .createWithDefault(1024) + val CONNECT_EXECUTE_REATTACHABLE_ENABLED = +ConfigBuilder("spark.connect.execute.reattachable.enabled") + .internal() + .doc("Enables reattachable execution on the server. If disabled and a client requests it, " + +"non-reattachable execution will follow and should run until query completion. This will " + +"work, unless there is a GRPC stream error, in which case the client will discover that " + +"execution is not reattachable when trying to reattach fails.") + .version("3.5.0") + .booleanConf + .createWithDefault(true) + val CONNECT_EXECUTE_REATTACHABLE_SENDER_MAX_STREAM_DURATION = ConfigBuilder("spark.connect.execute.reattachable.senderMaxStreamDuration") .internal() @@ -76,7 +87,7 @@ object Connect { "Set to 0 for unlimited.") .version("3.5.0") .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString("5m") + .createWithDefaultString("2m") val CONNECT_EXECUTE_REATTACHABLE_SENDER_MAX_STREAM_SIZE = ConfigBuilder("spark.connect.execute.reattachable.senderMaxStreamSize") diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/CachedStreamResponse.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/CachedStreamResponse.scala index ec9fce785ba..a2bbe14f201 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/CachedStreamResponse.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/CachedStreamResponse.scala @@ -22,6 +22,8 @@ import com.google.protob
[spark] branch master updated: [SPARK-44709][CONNECT] Run ExecuteGrpcResponseSender in reattachable execute in new thread to fix flow control
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 418bba5ad60 [SPARK-44709][CONNECT] Run ExecuteGrpcResponseSender in reattachable execute in new thread to fix flow control 418bba5ad60 is described below commit 418bba5ad6053449a141f3c9c31ed3ad998995b8 Author: Juliusz Sompolski AuthorDate: Tue Aug 8 18:32:25 2023 +0200 [SPARK-44709][CONNECT] Run ExecuteGrpcResponseSender in reattachable execute in new thread to fix flow control ### What changes were proposed in this pull request? If executePlan / reattachExecute handling is done directly on the GRPC thread, flow control OnReady events are getting queued until after the handler returns, so OnReadyHandler never gets notified until after the handler exits. The correct way to use it is for the handler to delegate work to another thread and exit. See https://github.com/grpc/grpc-java/issues/7361 Tidied up and added a lot of logging and statistics to ExecuteGrpcResponseSender and ExecuteResponseObserver to be able to observe this behaviour. Followup work in https://issues.apache.org/jira/browse/SPARK-44625 is needed for cleanup of abandoned executions that will also make sure that these threads are joined. ### Why are the changes needed? ExecuteGrpcResponseSender gets stuck waiting on grpcCallObserverReadySignal because events from OnReadyHandler do not arrive. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added extensive debugging to ExecuteGrpcResponseSender and ExecuteResponseObserver and tested and observer the behaviour of all the threads. Closes #42355 from juliuszsompolski/spark-rpc-extra-thread. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell --- .../apache/spark/sql/connect/config/Connect.scala | 13 +- .../connect/execution/CachedStreamResponse.scala | 2 + .../execution/ExecuteGrpcResponseSender.scala | 164 +++-- .../execution/ExecuteResponseObserver.scala| 116 --- .../connect/execution/ExecuteThreadRunner.scala| 3 +- .../spark/sql/connect/service/ExecuteHolder.scala | 21 ++- .../service/SparkConnectExecutePlanHandler.scala | 20 +-- .../SparkConnectReattachExecuteHandler.scala | 22 +-- 8 files changed, 264 insertions(+), 97 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala index e25cb5cbab2..0be53064cc0 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala @@ -74,6 +74,17 @@ object Connect { .intConf .createWithDefault(1024) + val CONNECT_EXECUTE_REATTACHABLE_ENABLED = +ConfigBuilder("spark.connect.execute.reattachable.enabled") + .internal() + .doc("Enables reattachable execution on the server. If disabled and a client requests it, " + +"non-reattachable execution will follow and should run until query completion. This will " + +"work, unless there is a GRPC stream error, in which case the client will discover that " + +"execution is not reattachable when trying to reattach fails.") + .version("3.5.0") + .booleanConf + .createWithDefault(true) + val CONNECT_EXECUTE_REATTACHABLE_SENDER_MAX_STREAM_DURATION = ConfigBuilder("spark.connect.execute.reattachable.senderMaxStreamDuration") .internal() @@ -82,7 +93,7 @@ object Connect { "Set to 0 for unlimited.") .version("3.5.0") .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString("5m") + .createWithDefaultString("2m") val CONNECT_EXECUTE_REATTACHABLE_SENDER_MAX_STREAM_SIZE = ConfigBuilder("spark.connect.execute.reattachable.senderMaxStreamSize") diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/CachedStreamResponse.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/CachedStreamResponse.scala index ec9fce785ba..a2bbe14f201 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/CachedStreamResponse.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/CachedStreamResponse.scala @@ -22,6 +22,8 @@ import com.google.protobuf.MessageLite private[execution] case class CachedStreamResponse[T <: MessageLite]( // the actual cached respon
[spark] branch branch-3.5 updated: [SPARK-44715][CONNECT] Bring back callUdf and udf function
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 0c27cb702c7 [SPARK-44715][CONNECT] Bring back callUdf and udf function 0c27cb702c7 is described below commit 0c27cb702c7b41b2518517d16d6d4108c6841271 Author: Herman van Hovell AuthorDate: Tue Aug 8 15:41:36 2023 +0200 [SPARK-44715][CONNECT] Bring back callUdf and udf function ### What changes were proposed in this pull request? This PR adds the `udf` (with a return type), and `callUDF` functions to `functions.scala` for the Spark Connect Scala Client. ### Why are the changes needed? We want the Spark Connect Scala Client to be as compatible as possible with the existing sql/core APIs. ### Does this PR introduce _any_ user-facing change? Yes. It adds more exposed functions. ### How was this patch tested? Added tests to `UserDefinedFunctionE2ETestSuite` and `FunctionTestSuite`. I have also updated the compatibility checks. Closes #42387 from hvanhovell/SPARK-44715. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 8c444f497137d5abb3a94b576ec0fea55dc18bbc) Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/functions.scala | 40 ++ .../org/apache/spark/sql/FunctionTestSuite.scala | 2 ++ .../sql/UserDefinedFunctionE2ETestSuite.scala | 20 +++ .../CheckConnectJvmClientCompatibility.scala | 7 4 files changed, 62 insertions(+), 7 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala index 89bfc998179..fa8c5782e06 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala @@ -8056,6 +8056,46 @@ object functions { } // scalastyle:off line.size.limit + /** + * Defines a deterministic user-defined function (UDF) using a Scala closure. For this variant, + * the caller must specify the output data type, and there is no automatic input type coercion. + * By default the returned UDF is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. + * + * Note that, although the Scala closure can have primitive-type function argument, it doesn't + * work well with null values. Because the Scala closure is passed in as Any type, there is no + * type information for the function arguments. Without the type information, Spark may blindly + * pass null to the Scala closure with primitive-type argument, and the closure will see the + * default value of the Java type for the null argument, e.g. `udf((x: Int) => x, IntegerType)`, + * the result is 0 for null input. + * + * @param f + * A closure in Scala + * @param dataType + * The output data type of the UDF + * + * @group udf_funcs + * @since 3.5.0 + */ + @deprecated( +"Scala `udf` method with return type parameter is deprecated. " + + "Please use Scala `udf` method without return type parameter.", +"3.0.0") + def udf(f: AnyRef, dataType: DataType): UserDefinedFunction = { +ScalarUserDefinedFunction(f, dataType) + } + + /** + * Call an user-defined function. + * + * @group udf_funcs + * @since 3.5.0 + */ + @scala.annotation.varargs + @deprecated("Use call_udf") + def callUDF(udfName: String, cols: Column*): Column = +call_function(udfName, cols: _*) + /** * Call an user-defined function. Example: * {{{ diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/FunctionTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/FunctionTestSuite.scala index 32004b6bcc1..4a8e108357f 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/FunctionTestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/FunctionTestSuite.scala @@ -249,6 +249,8 @@ class FunctionTestSuite extends ConnectFunSuite { pbFn.to_protobuf(a, "FakeMessage", "fakeBytes".getBytes(), Map.empty[String, String].asJava), pbFn.to_protobuf(a, "FakeMessage", "fakeBytes".getBytes())) + testEquals("call_udf", callUDF("bob", lit(1)), call_udf("bob", lit(1))) + test("assert_true no message") { val e = assert_true(a).expr assert(e.hasUnresolvedFunction) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala b/connector/connect/c
[spark] branch master updated: [SPARK-44715][CONNECT] Bring back callUdf and udf function
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8c444f49713 [SPARK-44715][CONNECT] Bring back callUdf and udf function 8c444f49713 is described below commit 8c444f497137d5abb3a94b576ec0fea55dc18bbc Author: Herman van Hovell AuthorDate: Tue Aug 8 15:41:36 2023 +0200 [SPARK-44715][CONNECT] Bring back callUdf and udf function ### What changes were proposed in this pull request? This PR adds the `udf` (with a return type), and `callUDF` functions to `functions.scala` for the Spark Connect Scala Client. ### Why are the changes needed? We want the Spark Connect Scala Client to be as compatible as possible with the existing sql/core APIs. ### Does this PR introduce _any_ user-facing change? Yes. It adds more exposed functions. ### How was this patch tested? Added tests to `UserDefinedFunctionE2ETestSuite` and `FunctionTestSuite`. I have also updated the compatibility checks. Closes #42387 from hvanhovell/SPARK-44715. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/functions.scala | 40 ++ .../org/apache/spark/sql/FunctionTestSuite.scala | 2 ++ .../sql/UserDefinedFunctionE2ETestSuite.scala | 20 +++ .../CheckConnectJvmClientCompatibility.scala | 7 4 files changed, 62 insertions(+), 7 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala index 89bfc998179..fa8c5782e06 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala @@ -8056,6 +8056,46 @@ object functions { } // scalastyle:off line.size.limit + /** + * Defines a deterministic user-defined function (UDF) using a Scala closure. For this variant, + * the caller must specify the output data type, and there is no automatic input type coercion. + * By default the returned UDF is deterministic. To change it to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. + * + * Note that, although the Scala closure can have primitive-type function argument, it doesn't + * work well with null values. Because the Scala closure is passed in as Any type, there is no + * type information for the function arguments. Without the type information, Spark may blindly + * pass null to the Scala closure with primitive-type argument, and the closure will see the + * default value of the Java type for the null argument, e.g. `udf((x: Int) => x, IntegerType)`, + * the result is 0 for null input. + * + * @param f + * A closure in Scala + * @param dataType + * The output data type of the UDF + * + * @group udf_funcs + * @since 3.5.0 + */ + @deprecated( +"Scala `udf` method with return type parameter is deprecated. " + + "Please use Scala `udf` method without return type parameter.", +"3.0.0") + def udf(f: AnyRef, dataType: DataType): UserDefinedFunction = { +ScalarUserDefinedFunction(f, dataType) + } + + /** + * Call an user-defined function. + * + * @group udf_funcs + * @since 3.5.0 + */ + @scala.annotation.varargs + @deprecated("Use call_udf") + def callUDF(udfName: String, cols: Column*): Column = +call_function(udfName, cols: _*) + /** * Call an user-defined function. Example: * {{{ diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/FunctionTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/FunctionTestSuite.scala index 32004b6bcc1..4a8e108357f 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/FunctionTestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/FunctionTestSuite.scala @@ -249,6 +249,8 @@ class FunctionTestSuite extends ConnectFunSuite { pbFn.to_protobuf(a, "FakeMessage", "fakeBytes".getBytes(), Map.empty[String, String].asJava), pbFn.to_protobuf(a, "FakeMessage", "fakeBytes".getBytes())) + testEquals("call_udf", callUDF("bob", lit(1)), call_udf("bob", lit(1))) + test("assert_true no message") { val e = assert_true(a).expr assert(e.hasUnresolvedFunction) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala index 258fa1e7c74..3a931c9a6ba 10064
[spark] branch branch-3.5 updated: [SPARK-44710][CONNECT] Add Dataset.dropDuplicatesWithinWatermark to Spark Connect Scala Client
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 08d526d13d6 [SPARK-44710][CONNECT] Add Dataset.dropDuplicatesWithinWatermark to Spark Connect Scala Client 08d526d13d6 is described below commit 08d526d13d69e08657a0158b7b0018aa3b1e6d74 Author: Herman van Hovell AuthorDate: Tue Aug 8 15:05:18 2023 +0200 [SPARK-44710][CONNECT] Add Dataset.dropDuplicatesWithinWatermark to Spark Connect Scala Client ### What changes were proposed in this pull request? This PR adds `Dataset.dropDuplicatesWithinWatermark` to the Spark Connect Scala Client. ### Why are the changes needed? Increase compatibility with the current sql/core APIs. ### Does this PR introduce _any_ user-facing change? Yes. It adds a new method to the scala client. ### How was this patch tested? Added a new (rudimentary) test to `ClientStreamingQuerySuite`. Closes #42384 from hvanhovell/SPARK-44710. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 93af0848e467fe4d58c0fb1242b738931390d6f8) Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 39 +++-- .../org/apache/spark/sql/ClientE2ETestSuite.scala | 20 +++ .../resources/query-tests/queries/distinct.json| 3 +- .../query-tests/queries/distinct.proto.bin | Bin 50 -> 52 bytes .../query-tests/queries/dropDuplicates.json| 3 +- .../query-tests/queries/dropDuplicates.proto.bin | Bin 50 -> 52 bytes .../queries/dropDuplicates_names_array.json| 3 +- .../queries/dropDuplicates_names_array.proto.bin | Bin 55 -> 57 bytes .../queries/dropDuplicates_names_seq.json | 3 +- .../queries/dropDuplicates_names_seq.proto.bin | Bin 54 -> 56 bytes .../queries/dropDuplicates_varargs.json| 3 +- .../queries/dropDuplicates_varargs.proto.bin | Bin 58 -> 60 bytes 12 files changed, 51 insertions(+), 23 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 8a7dce3987a..5f263903c8b 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2399,6 +2399,19 @@ class Dataset[T] private[sql] ( .addAllColumnNames(cols.asJava) } + private def buildDropDuplicates( + columns: Option[Seq[String]], + withinWaterMark: Boolean): Dataset[T] = sparkSession.newDataset(encoder) { builder => +val dropBuilder = builder.getDeduplicateBuilder + .setInput(plan.getRoot) + .setWithinWatermark(withinWaterMark) +if (columns.isDefined) { + dropBuilder.addAllColumnNames(columns.get.asJava) +} else { + dropBuilder.setAllColumnsAsKeys(true) +} + } + /** * Returns a new Dataset that contains only the unique rows from this Dataset. This is an alias * for `distinct`. @@ -2406,11 +2419,7 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def dropDuplicates(): Dataset[T] = sparkSession.newDataset(encoder) { builder => -builder.getDeduplicateBuilder - .setInput(plan.getRoot) - .setAllColumnsAsKeys(true) - } + def dropDuplicates(): Dataset[T] = buildDropDuplicates(None, withinWaterMark = false) /** * (Scala-specific) Returns a new Dataset with duplicate rows removed, considering only the @@ -2419,11 +2428,8 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def dropDuplicates(colNames: Seq[String]): Dataset[T] = sparkSession.newDataset(encoder) { -builder => - builder.getDeduplicateBuilder -.setInput(plan.getRoot) -.addAllColumnNames(colNames.asJava) + def dropDuplicates(colNames: Seq[String]): Dataset[T] = { +buildDropDuplicates(Option(colNames), withinWaterMark = false) } /** @@ -2443,16 +2449,14 @@ class Dataset[T] private[sql] ( */ @scala.annotation.varargs def dropDuplicates(col1: String, cols: String*): Dataset[T] = { -val colNames: Seq[String] = col1 +: cols -dropDuplicates(colNames) +dropDuplicates(col1 +: cols) } - def dropDuplicatesWithinWatermark(): Dataset[T] = { -dropDuplicatesWithinWatermark(this.columns) - } + def dropDuplicatesWithinWatermark(): Dataset[T] = +buildDropDuplicates(None, withinWaterMark = true) def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = { -throw new UnsupportedOperationException("dropDuplicatesWithinWatermark is not implemented.") +buildDropDuplicates(Option(colNames), wit
[spark] branch master updated: [SPARK-44710][CONNECT] Add Dataset.dropDuplicatesWithinWatermark to Spark Connect Scala Client
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 93af0848e46 [SPARK-44710][CONNECT] Add Dataset.dropDuplicatesWithinWatermark to Spark Connect Scala Client 93af0848e46 is described below commit 93af0848e467fe4d58c0fb1242b738931390d6f8 Author: Herman van Hovell AuthorDate: Tue Aug 8 15:05:18 2023 +0200 [SPARK-44710][CONNECT] Add Dataset.dropDuplicatesWithinWatermark to Spark Connect Scala Client ### What changes were proposed in this pull request? This PR adds `Dataset.dropDuplicatesWithinWatermark` to the Spark Connect Scala Client. ### Why are the changes needed? Increase compatibility with the current sql/core APIs. ### Does this PR introduce _any_ user-facing change? Yes. It adds a new method to the scala client. ### How was this patch tested? Added a new (rudimentary) test to `ClientStreamingQuerySuite`. Closes #42384 from hvanhovell/SPARK-44710. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 39 +++-- .../org/apache/spark/sql/ClientE2ETestSuite.scala | 20 +++ .../resources/query-tests/queries/distinct.json| 3 +- .../query-tests/queries/distinct.proto.bin | Bin 50 -> 52 bytes .../query-tests/queries/dropDuplicates.json| 3 +- .../query-tests/queries/dropDuplicates.proto.bin | Bin 50 -> 52 bytes .../queries/dropDuplicates_names_array.json| 3 +- .../queries/dropDuplicates_names_array.proto.bin | Bin 55 -> 57 bytes .../queries/dropDuplicates_names_seq.json | 3 +- .../queries/dropDuplicates_names_seq.proto.bin | Bin 54 -> 56 bytes .../queries/dropDuplicates_varargs.json| 3 +- .../queries/dropDuplicates_varargs.proto.bin | Bin 58 -> 60 bytes 12 files changed, 51 insertions(+), 23 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 8a7dce3987a..5f263903c8b 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2399,6 +2399,19 @@ class Dataset[T] private[sql] ( .addAllColumnNames(cols.asJava) } + private def buildDropDuplicates( + columns: Option[Seq[String]], + withinWaterMark: Boolean): Dataset[T] = sparkSession.newDataset(encoder) { builder => +val dropBuilder = builder.getDeduplicateBuilder + .setInput(plan.getRoot) + .setWithinWatermark(withinWaterMark) +if (columns.isDefined) { + dropBuilder.addAllColumnNames(columns.get.asJava) +} else { + dropBuilder.setAllColumnsAsKeys(true) +} + } + /** * Returns a new Dataset that contains only the unique rows from this Dataset. This is an alias * for `distinct`. @@ -2406,11 +2419,7 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def dropDuplicates(): Dataset[T] = sparkSession.newDataset(encoder) { builder => -builder.getDeduplicateBuilder - .setInput(plan.getRoot) - .setAllColumnsAsKeys(true) - } + def dropDuplicates(): Dataset[T] = buildDropDuplicates(None, withinWaterMark = false) /** * (Scala-specific) Returns a new Dataset with duplicate rows removed, considering only the @@ -2419,11 +2428,8 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def dropDuplicates(colNames: Seq[String]): Dataset[T] = sparkSession.newDataset(encoder) { -builder => - builder.getDeduplicateBuilder -.setInput(plan.getRoot) -.addAllColumnNames(colNames.asJava) + def dropDuplicates(colNames: Seq[String]): Dataset[T] = { +buildDropDuplicates(Option(colNames), withinWaterMark = false) } /** @@ -2443,16 +2449,14 @@ class Dataset[T] private[sql] ( */ @scala.annotation.varargs def dropDuplicates(col1: String, cols: String*): Dataset[T] = { -val colNames: Seq[String] = col1 +: cols -dropDuplicates(colNames) +dropDuplicates(col1 +: cols) } - def dropDuplicatesWithinWatermark(): Dataset[T] = { -dropDuplicatesWithinWatermark(this.columns) - } + def dropDuplicatesWithinWatermark(): Dataset[T] = +buildDropDuplicates(None, withinWaterMark = true) def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = { -throw new UnsupportedOperationException("dropDuplicatesWithinWatermark is not implemented.") +buildDropDuplicates(Option(colNames), withinWaterMark = true) } def dropDuplicatesWithinWatermark(colNames: Array[String]): Dataset[T] = { @@ -2461,8 +
[spark] branch branch-3.5 updated: [SPARK-44713][CONNECT][SQL] Move shared classes to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 57554e3f081 [SPARK-44713][CONNECT][SQL] Move shared classes to sql/api 57554e3f081 is described below commit 57554e3f081bbfa94ed5b46ff72616cc18e59da1 Author: Herman van Hovell AuthorDate: Tue Aug 8 15:04:07 2023 +0200 [SPARK-44713][CONNECT][SQL] Move shared classes to sql/api ### What changes were proposed in this pull request? This PR deduplicates the following classes: - `org.apache.spark.sql.SaveMode` - `org.apache.spark.api.java.function.FlatMapGroupsWithStateFunction` - `org.apache.spark.api.java.function.MapGroupsWithStateFunction` - `org.apache.spark.sql.streaming.GroupState` These classes were all duplicates in the Scala Client. I have moved the original versions to `sql/api` and I removed the connect equivalents. ### Why are the changes needed? Duplication is always good :). ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Compilation. Closes #42386 from hvanhovell/SPARK-44713. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit c46d4caa59865e9b99e02f6adc79f49f9ebc8f7f) Signed-off-by: Herman van Hovell --- .../main/java/org/apache/spark/sql/SaveMode.java | 58 .../function/FlatMapGroupsWithStateFunction.java | 39 --- .../java/function/MapGroupsWithStateFunction.java | 38 --- .../apache/spark/sql/streaming/GroupState.scala| 336 - project/MimaExcludes.scala | 13 +- .../function/FlatMapGroupsWithStateFunction.java | 0 .../java/function/MapGroupsWithStateFunction.java | 0 .../main/java/org/apache/spark/sql/SaveMode.java | 0 .../apache/spark/sql/streaming/GroupState.scala| 0 9 files changed, 6 insertions(+), 478 deletions(-) diff --git a/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/SaveMode.java b/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/SaveMode.java deleted file mode 100644 index 95af157687c..000 --- a/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/SaveMode.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - *http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql; - -import org.apache.spark.annotation.Stable; - -/** - * SaveMode is used to specify the expected behavior of saving a DataFrame to a data source. - * - * @since 3.4.0 - */ -@Stable -public enum SaveMode { - /** - * Append mode means that when saving a DataFrame to a data source, if data/table already exists, - * contents of the DataFrame are expected to be appended to existing data. - * - * @since 3.4.0 - */ - Append, - /** - * Overwrite mode means that when saving a DataFrame to a data source, - * if data/table already exists, existing data is expected to be overwritten by the contents of - * the DataFrame. - * - * @since 3.4.0 - */ - Overwrite, - /** - * ErrorIfExists mode means that when saving a DataFrame to a data source, if data already exists, - * an exception is expected to be thrown. - * - * @since 3.4.0 - */ - ErrorIfExists, - /** - * Ignore mode means that when saving a DataFrame to a data source, if data already exists, - * the save operation is expected to not save the contents of the DataFrame and to not - * change the existing data. - * - * @since 3.4.0 - */ - Ignore -} diff --git a/connector/connect/common/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java b/connector/connect/common/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java deleted file mode 100644 index c917c8d28be..000 --- a/connector/connect/common/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - *
[spark] branch master updated: [SPARK-44713][CONNECT][SQL] Move shared classes to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c46d4caa598 [SPARK-44713][CONNECT][SQL] Move shared classes to sql/api c46d4caa598 is described below commit c46d4caa59865e9b99e02f6adc79f49f9ebc8f7f Author: Herman van Hovell AuthorDate: Tue Aug 8 15:04:07 2023 +0200 [SPARK-44713][CONNECT][SQL] Move shared classes to sql/api ### What changes were proposed in this pull request? This PR deduplicates the following classes: - `org.apache.spark.sql.SaveMode` - `org.apache.spark.api.java.function.FlatMapGroupsWithStateFunction` - `org.apache.spark.api.java.function.MapGroupsWithStateFunction` - `org.apache.spark.sql.streaming.GroupState` These classes were all duplicates in the Scala Client. I have moved the original versions to `sql/api` and I removed the connect equivalents. ### Why are the changes needed? Duplication is always good :). ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Compilation. Closes #42386 from hvanhovell/SPARK-44713. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../main/java/org/apache/spark/sql/SaveMode.java | 58 .../function/FlatMapGroupsWithStateFunction.java | 39 --- .../java/function/MapGroupsWithStateFunction.java | 38 --- .../apache/spark/sql/streaming/GroupState.scala| 336 - project/MimaExcludes.scala | 13 +- .../function/FlatMapGroupsWithStateFunction.java | 0 .../java/function/MapGroupsWithStateFunction.java | 0 .../main/java/org/apache/spark/sql/SaveMode.java | 0 .../apache/spark/sql/streaming/GroupState.scala| 0 9 files changed, 6 insertions(+), 478 deletions(-) diff --git a/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/SaveMode.java b/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/SaveMode.java deleted file mode 100644 index 95af157687c..000 --- a/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/SaveMode.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - *http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql; - -import org.apache.spark.annotation.Stable; - -/** - * SaveMode is used to specify the expected behavior of saving a DataFrame to a data source. - * - * @since 3.4.0 - */ -@Stable -public enum SaveMode { - /** - * Append mode means that when saving a DataFrame to a data source, if data/table already exists, - * contents of the DataFrame are expected to be appended to existing data. - * - * @since 3.4.0 - */ - Append, - /** - * Overwrite mode means that when saving a DataFrame to a data source, - * if data/table already exists, existing data is expected to be overwritten by the contents of - * the DataFrame. - * - * @since 3.4.0 - */ - Overwrite, - /** - * ErrorIfExists mode means that when saving a DataFrame to a data source, if data already exists, - * an exception is expected to be thrown. - * - * @since 3.4.0 - */ - ErrorIfExists, - /** - * Ignore mode means that when saving a DataFrame to a data source, if data already exists, - * the save operation is expected to not save the contents of the DataFrame and to not - * change the existing data. - * - * @since 3.4.0 - */ - Ignore -} diff --git a/connector/connect/common/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java b/connector/connect/common/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java deleted file mode 100644 index c917c8d28be..000 --- a/connector/connect/common/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding
[spark] branch branch-3.5 updated: [SPARK-43429][CONNECT] Add Default & Active SparkSession for Scala Client
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 5a71a2e4a3d [SPARK-43429][CONNECT] Add Default & Active SparkSession for Scala Client 5a71a2e4a3d is described below commit 5a71a2e4a3d6ad5c6393b64fb76f571051ee3c94 Author: Herman van Hovell AuthorDate: Tue Aug 8 04:15:07 2023 +0200 [SPARK-43429][CONNECT] Add Default & Active SparkSession for Scala Client ### What changes were proposed in this pull request? This adds the `default` and `active` session variables to `SparkSession`: - `default` session is global value. It is typically the first session created through `getOrCreate`. It can be changed through `set` or `clear`. If the session is closed and it is the `default` session we clear the `default` session. - `active` session is a thread local value. It is typically the first session created in this thread or it inherits is value from its parent thread. It can be changed through `set` or `clear`, please note that these methods operate thread locally, so they won't change the parent or children. If the session is closed and it is the `active` session for the current thread then we clear the active value (only for the current thread!). ### Why are the changes needed? To increase compatibility with the existing SparkSession API in `sql/core`. ### Does this PR introduce _any_ user-facing change? Yes. It adds a couple methods that were missing from the Scala Client. ### How was this patch tested? Added tests to `SparkSessionSuite`. Closes #42367 from hvanhovell/SPARK-43429. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 7493c5764f9644878babacccd4f688fe13ef84aa) Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 100 -- .../org/apache/spark/sql/SparkSessionSuite.scala | 144 +++-- .../CheckConnectJvmClientCompatibility.scala | 2 - 3 files changed, 225 insertions(+), 21 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 355d7edadc7..7367ed153f7 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import java.io.Closeable import java.net.URI import java.util.concurrent.TimeUnit._ -import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.atomic.{AtomicLong, AtomicReference} import scala.collection.JavaConverters._ import scala.reflect.runtime.universe.TypeTag @@ -730,6 +730,23 @@ object SparkSession extends Logging { override def load(c: Configuration): SparkSession = create(c) }) + /** The active SparkSession for the current thread. */ + private val activeThreadSession = new InheritableThreadLocal[SparkSession] + + /** Reference to the root SparkSession. */ + private val defaultSession = new AtomicReference[SparkSession] + + /** + * Set the (global) default [[SparkSession]], and (thread-local) active [[SparkSession]] when + * they are not set yet. + */ + private def setDefaultAndActiveSession(session: SparkSession): Unit = { +defaultSession.compareAndSet(null, session) +if (getActiveSession.isEmpty) { + setActiveSession(session) +} + } + /** * Create a new [[SparkSession]] based on the connect client [[Configuration]]. */ @@ -742,8 +759,17 @@ object SparkSession extends Logging { */ private[sql] def onSessionClose(session: SparkSession): Unit = { sessions.invalidate(session.client.configuration) +defaultSession.compareAndSet(session, null) +if (getActiveSession.contains(session)) { + clearActiveSession() +} } + /** + * Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]]. + * + * @since 3.4.0 + */ def builder(): Builder = new Builder() private[sql] lazy val cleaner = { @@ -799,10 +825,15 @@ object SparkSession extends Logging { * * This will always return a newly created session. * + * This method will update the default and/or active session if they are not set. + * * @since 3.5.0 */ def create(): SparkSession = { - tryCreateSessionFromClient().getOrElse(SparkSession.this.create(builder.configuration)) + val session = tryCreateSessionFromClient() +.getOrElse(SparkSession.this.create(builder.configuration)) + setDefaultAndActiveSession(session) + session } /** @@ -811,30 +842,79 @@ object SparkSe
[spark] branch master updated: [SPARK-43429][CONNECT] Add Default & Active SparkSession for Scala Client
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7493c5764f9 [SPARK-43429][CONNECT] Add Default & Active SparkSession for Scala Client 7493c5764f9 is described below commit 7493c5764f9644878babacccd4f688fe13ef84aa Author: Herman van Hovell AuthorDate: Tue Aug 8 04:15:07 2023 +0200 [SPARK-43429][CONNECT] Add Default & Active SparkSession for Scala Client ### What changes were proposed in this pull request? This adds the `default` and `active` session variables to `SparkSession`: - `default` session is global value. It is typically the first session created through `getOrCreate`. It can be changed through `set` or `clear`. If the session is closed and it is the `default` session we clear the `default` session. - `active` session is a thread local value. It is typically the first session created in this thread or it inherits is value from its parent thread. It can be changed through `set` or `clear`, please note that these methods operate thread locally, so they won't change the parent or children. If the session is closed and it is the `active` session for the current thread then we clear the active value (only for the current thread!). ### Why are the changes needed? To increase compatibility with the existing SparkSession API in `sql/core`. ### Does this PR introduce _any_ user-facing change? Yes. It adds a couple methods that were missing from the Scala Client. ### How was this patch tested? Added tests to `SparkSessionSuite`. Closes #42367 from hvanhovell/SPARK-43429. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 100 -- .../org/apache/spark/sql/SparkSessionSuite.scala | 144 +++-- .../CheckConnectJvmClientCompatibility.scala | 2 - 3 files changed, 225 insertions(+), 21 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 355d7edadc7..7367ed153f7 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import java.io.Closeable import java.net.URI import java.util.concurrent.TimeUnit._ -import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.atomic.{AtomicLong, AtomicReference} import scala.collection.JavaConverters._ import scala.reflect.runtime.universe.TypeTag @@ -730,6 +730,23 @@ object SparkSession extends Logging { override def load(c: Configuration): SparkSession = create(c) }) + /** The active SparkSession for the current thread. */ + private val activeThreadSession = new InheritableThreadLocal[SparkSession] + + /** Reference to the root SparkSession. */ + private val defaultSession = new AtomicReference[SparkSession] + + /** + * Set the (global) default [[SparkSession]], and (thread-local) active [[SparkSession]] when + * they are not set yet. + */ + private def setDefaultAndActiveSession(session: SparkSession): Unit = { +defaultSession.compareAndSet(null, session) +if (getActiveSession.isEmpty) { + setActiveSession(session) +} + } + /** * Create a new [[SparkSession]] based on the connect client [[Configuration]]. */ @@ -742,8 +759,17 @@ object SparkSession extends Logging { */ private[sql] def onSessionClose(session: SparkSession): Unit = { sessions.invalidate(session.client.configuration) +defaultSession.compareAndSet(session, null) +if (getActiveSession.contains(session)) { + clearActiveSession() +} } + /** + * Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]]. + * + * @since 3.4.0 + */ def builder(): Builder = new Builder() private[sql] lazy val cleaner = { @@ -799,10 +825,15 @@ object SparkSession extends Logging { * * This will always return a newly created session. * + * This method will update the default and/or active session if they are not set. + * * @since 3.5.0 */ def create(): SparkSession = { - tryCreateSessionFromClient().getOrElse(SparkSession.this.create(builder.configuration)) + val session = tryCreateSessionFromClient() +.getOrElse(SparkSession.this.create(builder.configuration)) + setDefaultAndActiveSession(session) + session } /** @@ -811,30 +842,79 @@ object SparkSession extends Logging { * If a session exist with the same configuration that is returned instead of c
[spark] branch branch-3.5 updated: [SPARK-44692][CONNECT][SQL] Move Trigger(s) to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new d8f02274c38 [SPARK-44692][CONNECT][SQL] Move Trigger(s) to sql/api d8f02274c38 is described below commit d8f02274c38c027e2d56f5158ce63f6e74255d2d Author: Herman van Hovell AuthorDate: Tue Aug 8 00:42:13 2023 +0200 [SPARK-44692][CONNECT][SQL] Move Trigger(s) to sql/api This PR moves `Triggers.scala` and `Trigger.scala` from `sql/core` to `sql/api`, and it removes the duplicates from the connect scala client. Not really needed, just some deduplication. No. Existing tests. Closes #42368 from hvanhovell/SPARK-44692. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 4eea89d339649152a1afcd8b7a32020454e71d42) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/streaming/Trigger.java| 180 - dev/checkstyle-suppressions.xml| 4 +- project/MimaExcludes.scala | 4 +- .../org/apache/spark/sql/streaming/Trigger.java| 0 .../spark/sql/execution/streaming/Triggers.scala | 6 +- .../spark/sql/execution/streaming/Triggers.scala | 113 - 6 files changed, 6 insertions(+), 301 deletions(-) diff --git a/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/streaming/Trigger.java b/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/streaming/Trigger.java deleted file mode 100644 index 27ffe67d990..000 --- a/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/streaming/Trigger.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - *http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.streaming; - -import java.util.concurrent.TimeUnit; - -import scala.concurrent.duration.Duration; - -import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.execution.streaming.AvailableNowTrigger$; -import org.apache.spark.sql.execution.streaming.ContinuousTrigger; -import org.apache.spark.sql.execution.streaming.OneTimeTrigger$; -import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger; - -/** - * Policy used to indicate how often results should be produced by a [[StreamingQuery]]. - * - * @since 3.5.0 - */ -@Evolving -public class Trigger { - // This is a copy of the same class in sql/core/.../streaming/Trigger.java - - /** - * A trigger policy that runs a query periodically based on an interval in processing time. - * If `interval` is 0, the query will run as fast as possible. - * - * @since 3.5.0 - */ - public static Trigger ProcessingTime(long intervalMs) { - return ProcessingTimeTrigger.create(intervalMs, TimeUnit.MILLISECONDS); - } - - /** - * (Java-friendly) - * A trigger policy that runs a query periodically based on an interval in processing time. - * If `interval` is 0, the query will run as fast as possible. - * - * {{{ - *import java.util.concurrent.TimeUnit - *df.writeStream().trigger(Trigger.ProcessingTime(10, TimeUnit.SECONDS)) - * }}} - * - * @since 3.5.0 - */ - public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) { - return ProcessingTimeTrigger.create(interval, timeUnit); - } - - /** - * (Scala-friendly) - * A trigger policy that runs a query periodically based on an interval in processing time. - * If `duration` is 0, the query will run as fast as possible. - * - * {{{ - *import scala.concurrent.duration._ - *df.writeStream.trigger(Trigger.ProcessingTime(10.seconds)) - * }}} - * @since 3.5.0 - */ - public static Trigger ProcessingTime(Duration interval) { - return ProcessingTimeTrigger.apply(interval); - } - - /** - * A trigger policy that runs a query periodically based on an interval in processing time. - * If `interval` is effectively 0, the query will run as fast as possible. - * - * {{{ - *df.writeStream.trigger(Trigger.ProcessingTime("10 seconds")
[spark] branch master updated: [SPARK-44692][CONNECT][SQL] Move Trigger(s) to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 4eea89d3396 [SPARK-44692][CONNECT][SQL] Move Trigger(s) to sql/api 4eea89d3396 is described below commit 4eea89d339649152a1afcd8b7a32020454e71d42 Author: Herman van Hovell AuthorDate: Tue Aug 8 00:42:13 2023 +0200 [SPARK-44692][CONNECT][SQL] Move Trigger(s) to sql/api ### What changes were proposed in this pull request? This PR moves `Triggers.scala` and `Trigger.scala` from `sql/core` to `sql/api`, and it removes the duplicates from the connect scala client. ### Why are the changes needed? Not really needed, just some deduplication. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #42368 from hvanhovell/SPARK-44692. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/streaming/Trigger.java| 180 - dev/checkstyle-suppressions.xml| 4 +- project/MimaExcludes.scala | 4 +- .../org/apache/spark/sql/streaming/Trigger.java| 0 .../spark/sql/execution/streaming/Triggers.scala | 6 +- .../spark/sql/execution/streaming/Triggers.scala | 113 - 6 files changed, 6 insertions(+), 301 deletions(-) diff --git a/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/streaming/Trigger.java b/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/streaming/Trigger.java deleted file mode 100644 index 27ffe67d990..000 --- a/connector/connect/client/jvm/src/main/java/org/apache/spark/sql/streaming/Trigger.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - *http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.streaming; - -import java.util.concurrent.TimeUnit; - -import scala.concurrent.duration.Duration; - -import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.execution.streaming.AvailableNowTrigger$; -import org.apache.spark.sql.execution.streaming.ContinuousTrigger; -import org.apache.spark.sql.execution.streaming.OneTimeTrigger$; -import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger; - -/** - * Policy used to indicate how often results should be produced by a [[StreamingQuery]]. - * - * @since 3.5.0 - */ -@Evolving -public class Trigger { - // This is a copy of the same class in sql/core/.../streaming/Trigger.java - - /** - * A trigger policy that runs a query periodically based on an interval in processing time. - * If `interval` is 0, the query will run as fast as possible. - * - * @since 3.5.0 - */ - public static Trigger ProcessingTime(long intervalMs) { - return ProcessingTimeTrigger.create(intervalMs, TimeUnit.MILLISECONDS); - } - - /** - * (Java-friendly) - * A trigger policy that runs a query periodically based on an interval in processing time. - * If `interval` is 0, the query will run as fast as possible. - * - * {{{ - *import java.util.concurrent.TimeUnit - *df.writeStream().trigger(Trigger.ProcessingTime(10, TimeUnit.SECONDS)) - * }}} - * - * @since 3.5.0 - */ - public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) { - return ProcessingTimeTrigger.create(interval, timeUnit); - } - - /** - * (Scala-friendly) - * A trigger policy that runs a query periodically based on an interval in processing time. - * If `duration` is 0, the query will run as fast as possible. - * - * {{{ - *import scala.concurrent.duration._ - *df.writeStream.trigger(Trigger.ProcessingTime(10.seconds)) - * }}} - * @since 3.5.0 - */ - public static Trigger ProcessingTime(Duration interval) { - return ProcessingTimeTrigger.apply(interval); - } - - /** - * A trigger policy that runs a query periodically based on an interval in processing time. - * If `interval` is effectively 0, the query will run as fast as possible. - * - * {{{ - *df
[spark] branch branch-3.4 updated: [SPARK-44634][SQL][3.4] Encoders.bean does no longer support nested beans with type arguments
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.4 by this push: new 1f5d78b5952 [SPARK-44634][SQL][3.4] Encoders.bean does no longer support nested beans with type arguments 1f5d78b5952 is described below commit 1f5d78b5952fcc6c7d36d3338a5594070e3a62dd Author: Giambattista Bloisi AuthorDate: Mon Aug 7 15:11:02 2023 +0200 [SPARK-44634][SQL][3.4] Encoders.bean does no longer support nested beans with type arguments ### What changes were proposed in this pull request? This is a port of [42327](https://github.com/apache/spark/pull/42327) This PR fixes a regression introduced in Spark 3.4.x where Encoders.bean is no longer able to process nested beans having type arguments. For example: ``` class A { T value; // value getter and setter } class B { A stringHolder; // stringHolder getter and setter } Encoders.bean(B.class); // throws "SparkUnsupportedOperationException: [ENCODER_NOT_FOUND]..." ``` ### Why are the changes needed? JavaTypeInference.encoderFor main match does not manage ParameterizedType and TypeVariable cases. I think this is a regression introduced after getting rid of usage of guava TypeToken: [SPARK-42093 SQL Move JavaTypeInference to AgnosticEncoders](https://github.com/apache/spark/commit/18672003513d5a4aa610b6b94dbbc15c33185d3#diff-1191737b908340a2f4c22b71b1c40ebaa0da9d8b40c958089c346a3bda26943b) hvanhovell cloud-fan In this PR I'm leveraging commons lang3 TypeUtils functionalities to solve ParameterizedType type arguments for classes ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests have been extended to check correct encoding of a nested bean having type arguments. Closes #42379 from gbloisi-openaire/spark-44634-branch-3.4. Authored-by: Giambattista Bloisi Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/JavaTypeInference.scala | 85 +- .../spark/sql/catalyst/JavaBeanWithGenerics.java | 41 +++ .../sql/catalyst/JavaTypeInferenceSuite.scala | 4 + 3 files changed, 65 insertions(+), 65 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 36b98737a20..75aca3ccbdd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -18,12 +18,14 @@ package org.apache.spark.sql.catalyst import java.beans.{Introspector, PropertyDescriptor} import java.lang.reflect.{ParameterizedType, Type, TypeVariable} -import java.util.{ArrayDeque, List => JList, Map => JMap} +import java.util.{List => JList, Map => JMap} import javax.annotation.Nonnull -import scala.annotation.tailrec +import scala.collection.JavaConverters._ import scala.reflect.ClassTag +import org.apache.commons.lang3.reflect.{TypeUtils => JavaTypeUtils} + import org.apache.spark.SPARK_DOC_ROOT import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, DayTimeIntervalEncoder, DEFAULT_JAVA_DECIMAL_ENCODER, EncoderField, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaEnumEncoder, LocalDateTimeEncoder, MapEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, P [...] @@ -58,7 +60,8 @@ object JavaTypeInference { encoderFor(beanType, Set.empty).asInstanceOf[AgnosticEncoder[T]] } - private def encoderFor(t: Type, seenTypeSet: Set[Class[_]]): AgnosticEncoder[_] = t match { + private def encoderFor(t: Type, seenTypeSet: Set[Class[_]], +typeVariables: Map[TypeVariable[_], Type] = Map.empty): AgnosticEncoder[_] = t match { case c: Class[_] if c == java.lang.Boolean.TYPE => PrimitiveBooleanEncoder case c: Class[_] if c == java.lang.Byte.TYPE => PrimitiveByteEncoder @@ -102,18 +105,24 @@ object JavaTypeInference { UDTEncoder(udt, udt.getClass) case c: Class[_] if c.isArray => - val elementEncoder = encoderFor(c.getComponentType, seenTypeSet) + val elementEncoder = encoderFor(c.getComponentType, seenTypeSet, typeVariables) ArrayEncoder(elementEncoder, elementEncoder.nullable) -case ImplementsList(c, Array(elementCls)) => - val element = encoderFor(elementCls, seenTypeSet) +case c: Class[
[spark] branch branch-3.5 updated: [SPARK-44686][CONNECT][SQL] Add the ability to create a RowEncoder in Encoders.scala
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 8f6f301fa77 [SPARK-44686][CONNECT][SQL] Add the ability to create a RowEncoder in Encoders.scala 8f6f301fa77 is described below commit 8f6f301fa778dfd0fd7dec4a29df7106846d3277 Author: Herman van Hovell AuthorDate: Mon Aug 7 15:09:58 2023 +0200 [SPARK-44686][CONNECT][SQL] Add the ability to create a RowEncoder in Encoders.scala ### What changes were proposed in this pull request? ### Why are the changes needed? It is currently not possible to create a `RowEncoder` using public API. The internal APIs for this will change in Spark 3.5, this means that library maintainers have to update their code if they use a RowEncoder. To avoid happening again, we add this method to the public API. ### Does this PR introduce _any_ user-facing change? Yes. It adds the `row` method to `Encoders`. ### How was this patch tested? Added tests to connect and sql. Closes #42366 from hvanhovell/SPARK-44686. Lead-authored-by: Herman van Hovell Co-authored-by: Hyukjin Kwon Signed-off-by: Herman van Hovell (cherry picked from commit bf7654998fbbec9d5bdee6f46462cffef495545f) Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Encoders.scala | 10 ++- .../org/apache/spark/sql/JavaEncoderSuite.java | 31 +++--- project/MimaExcludes.scala | 2 ++ .../main/java/org/apache/spark/sql/RowFactory.java | 0 .../main/scala/org/apache/spark/sql/Encoders.scala | 7 + .../org/apache/spark/sql/JavaDatasetSuite.java | 19 + 6 files changed, 64 insertions(+), 5 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala index 3f2f7ec96d4..74f01338031 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql import scala.reflect.runtime.universe.TypeTag import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection} -import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder +import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder => RowEncoderFactory} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ +import org.apache.spark.sql.types.StructType /** * Methods for creating an [[Encoder]]. @@ -168,6 +169,13 @@ object Encoders { */ def bean[T](beanClass: Class[T]): Encoder[T] = JavaTypeInference.encoderFor(beanClass) + /** + * Creates a [[Row]] encoder for schema `schema`. + * + * @since 3.5.0 + */ + def row(schema: StructType): Encoder[Row] = RowEncoderFactory.encoderFor(schema) + private def tupleEncoder[T](encoders: Encoder[_]*): Encoder[T] = { ProductEncoder.tuple(encoders.asInstanceOf[Seq[AgnosticEncoder[_]]]).asInstanceOf[Encoder[T]] } diff --git a/connector/connect/client/jvm/src/test/java/org/apache/spark/sql/JavaEncoderSuite.java b/connector/connect/client/jvm/src/test/java/org/apache/spark/sql/JavaEncoderSuite.java index c8210a7a485..6e5fb72d496 100644 --- a/connector/connect/client/jvm/src/test/java/org/apache/spark/sql/JavaEncoderSuite.java +++ b/connector/connect/client/jvm/src/test/java/org/apache/spark/sql/JavaEncoderSuite.java @@ -16,21 +16,26 @@ */ package org.apache.spark.sql; +import java.io.Serializable; +import java.math.BigDecimal; +import java.util.Arrays; +import java.util.List; + import org.junit.*; import static org.junit.Assert.*; import static org.apache.spark.sql.Encoders.*; import static org.apache.spark.sql.functions.*; +import static org.apache.spark.sql.RowFactory.create; import org.apache.spark.sql.connect.client.SparkConnectClient; import org.apache.spark.sql.connect.client.util.SparkConnectServerUtils; - -import java.math.BigDecimal; -import java.util.Arrays; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.types.StructType; /** * Tests for the encoders class. */ -public class JavaEncoderSuite { +public class JavaEncoderSuite implements Serializable { private static SparkSession spark; @BeforeClass @@ -91,4 +96,22 @@ public class JavaEncoderSuite { dataset(DECIMAL(), bigDec(1000, 2), bigDec(2, 2)) .select(sum(v)).as(DECIMAL()).head().setScale(2)); } + + @Test + public void testRowEncoder() { +final StructType schema = new StructType() +.add("a", "int") +.add("b", "string"); +final Dataset df = spark.range(3) +.map(ne
[spark] branch master updated: [SPARK-44686][CONNECT][SQL] Add the ability to create a RowEncoder in Encoders.scala
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new bf7654998fb [SPARK-44686][CONNECT][SQL] Add the ability to create a RowEncoder in Encoders.scala bf7654998fb is described below commit bf7654998fbbec9d5bdee6f46462cffef495545f Author: Herman van Hovell AuthorDate: Mon Aug 7 15:09:58 2023 +0200 [SPARK-44686][CONNECT][SQL] Add the ability to create a RowEncoder in Encoders.scala ### What changes were proposed in this pull request? ### Why are the changes needed? It is currently not possible to create a `RowEncoder` using public API. The internal APIs for this will change in Spark 3.5, this means that library maintainers have to update their code if they use a RowEncoder. To avoid happening again, we add this method to the public API. ### Does this PR introduce _any_ user-facing change? Yes. It adds the `row` method to `Encoders`. ### How was this patch tested? Added tests to connect and sql. Closes #42366 from hvanhovell/SPARK-44686. Lead-authored-by: Herman van Hovell Co-authored-by: Hyukjin Kwon Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Encoders.scala | 10 ++- .../org/apache/spark/sql/JavaEncoderSuite.java | 31 +++--- project/MimaExcludes.scala | 2 ++ .../main/java/org/apache/spark/sql/RowFactory.java | 0 .../main/scala/org/apache/spark/sql/Encoders.scala | 7 + .../org/apache/spark/sql/JavaDatasetSuite.java | 19 + 6 files changed, 64 insertions(+), 5 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala index 3f2f7ec96d4..74f01338031 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql import scala.reflect.runtime.universe.TypeTag import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection} -import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder +import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder => RowEncoderFactory} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ +import org.apache.spark.sql.types.StructType /** * Methods for creating an [[Encoder]]. @@ -168,6 +169,13 @@ object Encoders { */ def bean[T](beanClass: Class[T]): Encoder[T] = JavaTypeInference.encoderFor(beanClass) + /** + * Creates a [[Row]] encoder for schema `schema`. + * + * @since 3.5.0 + */ + def row(schema: StructType): Encoder[Row] = RowEncoderFactory.encoderFor(schema) + private def tupleEncoder[T](encoders: Encoder[_]*): Encoder[T] = { ProductEncoder.tuple(encoders.asInstanceOf[Seq[AgnosticEncoder[_]]]).asInstanceOf[Encoder[T]] } diff --git a/connector/connect/client/jvm/src/test/java/org/apache/spark/sql/JavaEncoderSuite.java b/connector/connect/client/jvm/src/test/java/org/apache/spark/sql/JavaEncoderSuite.java index c8210a7a485..6e5fb72d496 100644 --- a/connector/connect/client/jvm/src/test/java/org/apache/spark/sql/JavaEncoderSuite.java +++ b/connector/connect/client/jvm/src/test/java/org/apache/spark/sql/JavaEncoderSuite.java @@ -16,21 +16,26 @@ */ package org.apache.spark.sql; +import java.io.Serializable; +import java.math.BigDecimal; +import java.util.Arrays; +import java.util.List; + import org.junit.*; import static org.junit.Assert.*; import static org.apache.spark.sql.Encoders.*; import static org.apache.spark.sql.functions.*; +import static org.apache.spark.sql.RowFactory.create; import org.apache.spark.sql.connect.client.SparkConnectClient; import org.apache.spark.sql.connect.client.util.SparkConnectServerUtils; - -import java.math.BigDecimal; -import java.util.Arrays; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.types.StructType; /** * Tests for the encoders class. */ -public class JavaEncoderSuite { +public class JavaEncoderSuite implements Serializable { private static SparkSession spark; @BeforeClass @@ -91,4 +96,22 @@ public class JavaEncoderSuite { dataset(DECIMAL(), bigDec(1000, 2), bigDec(2, 2)) .select(sum(v)).as(DECIMAL()).head().setScale(2)); } + + @Test + public void testRowEncoder() { +final StructType schema = new StructType() +.add("a", "int") +.add("b", "string"); +final Dataset df = spark.range(3) +.map(new MapFunction() { + @Override + public Row call(Long i) { +
[spark] branch branch-3.5 updated: [SPARK-44634][SQL] Encoders.bean does no longer support nested beans with type arguments
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new e3b031276e4 [SPARK-44634][SQL] Encoders.bean does no longer support nested beans with type arguments e3b031276e4 is described below commit e3b031276e4ab626f7db7b8d95f01a598e25a6b1 Author: Giambattista Bloisi AuthorDate: Sun Aug 6 21:47:57 2023 +0200 [SPARK-44634][SQL] Encoders.bean does no longer support nested beans with type arguments ### What changes were proposed in this pull request? This PR fixes a regression introduced in Spark 3.4.x where Encoders.bean is no longer able to process nested beans having type arguments. For example: ``` class A { T value; // value getter and setter } class B { A stringHolder; // stringHolder getter and setter } Encoders.bean(B.class); // throws "SparkUnsupportedOperationException: [ENCODER_NOT_FOUND]..." ``` ### Why are the changes needed? JavaTypeInference.encoderFor main match does not manage ParameterizedType and TypeVariable cases. I think this is a regression introduced after getting rid of usage of guava TypeToken: [SPARK-42093 SQL Move JavaTypeInference to AgnosticEncoders](https://github.com/apache/spark/commit/18672003513d5a4aa610b6b94dbbc15c33185d3#diff-1191737b908340a2f4c22b71b1c40ebaa0da9d8b40c958089c346a3bda26943b) hvanhovell cloud-fan In this PR I'm leveraging commons lang3 TypeUtils functionalities to solve ParameterizedType type arguments for classes ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests have been extended to check correct encoding of a nested bean having type arguments. Closes #42327 from gbloisi-openaire/spark-44634. Authored-by: Giambattista Bloisi Signed-off-by: Herman van Hovell (cherry picked from commit d6998979427b6ad3a0f16d6966b3927d40440a60) Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/JavaTypeInference.scala | 84 +- .../spark/sql/catalyst/JavaBeanWithGenerics.java | 41 +++ .../sql/catalyst/JavaTypeInferenceSuite.scala | 4 ++ 3 files changed, 64 insertions(+), 65 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index f352d28a7b5..3d536b735db 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -18,12 +18,14 @@ package org.apache.spark.sql.catalyst import java.beans.{Introspector, PropertyDescriptor} import java.lang.reflect.{ParameterizedType, Type, TypeVariable} -import java.util.{ArrayDeque, List => JList, Map => JMap} +import java.util.{List => JList, Map => JMap} import javax.annotation.Nonnull -import scala.annotation.tailrec +import scala.collection.JavaConverters._ import scala.reflect.ClassTag +import org.apache.commons.lang3.reflect.{TypeUtils => JavaTypeUtils} + import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, DayTimeIntervalEncoder, DEFAULT_JAVA_DECIMAL_ENCODER, EncoderField, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaEnumEncoder, LocalDateTimeEncoder, MapEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, P [...] import org.apache.spark.sql.errors.ExecutionErrors @@ -57,7 +59,8 @@ object JavaTypeInference { encoderFor(beanType, Set.empty).asInstanceOf[AgnosticEncoder[T]] } - private def encoderFor(t: Type, seenTypeSet: Set[Class[_]]): AgnosticEncoder[_] = t match { + private def encoderFor(t: Type, seenTypeSet: Set[Class[_]], +typeVariables: Map[TypeVariable[_], Type] = Map.empty): AgnosticEncoder[_] = t match { case c: Class[_] if c == java.lang.Boolean.TYPE => PrimitiveBooleanEncoder case c: Class[_] if c == java.lang.Byte.TYPE => PrimitiveByteEncoder @@ -101,18 +104,24 @@ object JavaTypeInference { UDTEncoder(udt, udt.getClass) case c: Class[_] if c.isArray => - val elementEncoder = encoderFor(c.getComponentType, seenTypeSet) + val elementEncoder = encoderFor(c.getComponentType, seenTypeSet, typeVariables) ArrayEncoder(elementEncoder, elementEncoder.nullable) -case ImplementsList(c, Array(elementCls)) => - val element = encoderFor(elementCls, seenTypeSet) +case c: Class[
[spark] branch master updated (74ae1e3434c -> d6998979427)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 74ae1e3434c [SPARK-42500][SQL] ConstantPropagation support more case add d6998979427 [SPARK-44634][SQL] Encoders.bean does no longer support nested beans with type arguments No new revisions were added by this update. Summary of changes: .../spark/sql/catalyst/JavaTypeInference.scala | 84 +- .../spark/sql/catalyst/JavaBeanWithGenerics.java} | 29 .../sql/catalyst/JavaTypeInferenceSuite.scala | 4 ++ 3 files changed, 39 insertions(+), 78 deletions(-) copy sql/{hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java => catalyst/src/test/java/org/apache/spark/sql/catalyst/JavaBeanWithGenerics.java} (67%) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (6073d721933 -> 84ea6f242e4)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 6073d721933 [SPARK-44675][INFRA] Increase ReservedCodeCacheSize for release build add 84ea6f242e4 [SPARK-44656][CONNECT] Make all iterators CloseableIterators No new revisions were added by this update. Summary of changes: .../main/scala/org/apache/spark/sql/Dataset.scala | 4 +-- .../scala/org/apache/spark/sql/SparkSession.scala | 21 +--- .../sql/connect/client/CloseableIterator.scala | 40 +++--- .../client/CustomSparkConnectBlockingStub.scala| 10 -- .../ExecutePlanResponseReattachableIterator.scala | 34 ++ .../connect/client/GrpcExceptionConverter.scala| 10 -- .../sql/connect/client/GrpcRetryHandler.scala | 24 +++-- .../sql/connect/client/SparkConnectClient.scala| 7 ++-- .../spark/sql/connect/client/SparkResult.scala | 37 +++- .../connect/client/arrow/ArrowDeserializer.scala | 1 + .../connect/client/arrow/ArrowEncoderUtils.scala | 2 -- .../sql/connect/client/arrow/ArrowSerializer.scala | 1 + .../org/apache/spark/sql/ClientE2ETestSuite.scala | 2 +- .../connect/client/arrow/ArrowEncoderSuite.scala | 1 + 14 files changed, 107 insertions(+), 87 deletions(-) copy sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BucketingUtilsSuite.scala => connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala (52%) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.5 updated: [SPARK-44656][CONNECT] Make all iterators CloseableIterators
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new d18305d5e93 [SPARK-44656][CONNECT] Make all iterators CloseableIterators d18305d5e93 is described below commit d18305d5e9312a438317b9b2ff800f2c074e3917 Author: Juliusz Sompolski AuthorDate: Fri Aug 4 16:06:57 2023 +0200 [SPARK-44656][CONNECT] Make all iterators CloseableIterators ### What changes were proposed in this pull request? This makes sure that all iterators used in Spark Connect scala client are `CloseableIterator`. 1. Makes `CustomSparkConnectBlockingStub.executePlan` return `CloseableIterator` and make all wrappers respect that. 2. Makes `ExecutePlanResponseReattachableIterator` a `CloseableIterator`, with an implementation that will inform the server that query result can be released with ReleaseExecute. 3. Makes `SparkResult.iterator` explicitly a `CloseableIterator`, and also register the `SparkResult.responses` iterator as with the `SparkResultCloseable` cleaner, which will make it close upon GC, if not closed explicitly sooner. 4. Because `Dataset.toLocalIterator` requires a Java iterator, implement a conversion to `java.util.Iterator with AutoCloseable` to be returned there 5. Using `CloseableIterator` consistently everywhere else removes the need to convert between iterator types. ### Why are the changes needed? Properly closeable iterators are needed for resource management, and with reattachable execution to inform server that processing finished. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Exercise current E2E tests. Co-authored-by: Alice Sayutina Closes #42331 from juliuszsompolski/closeable_iterators. Lead-authored-by: Juliusz Sompolski Co-authored-by: Alice Sayutina Signed-off-by: Herman van Hovell (cherry picked from commit 84ea6f242e4982187edc0a8f5786e7dc69ec31d7) Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 4 +- .../scala/org/apache/spark/sql/SparkSession.scala | 21 -- .../sql/connect/client/CloseableIterator.scala | 46 ++ .../client/CustomSparkConnectBlockingStub.scala| 10 +++-- .../ExecutePlanResponseReattachableIterator.scala | 34 +--- .../connect/client/GrpcExceptionConverter.scala| 10 - .../sql/connect/client/GrpcRetryHandler.scala | 24 ++- .../sql/connect/client/SparkConnectClient.scala| 7 ++-- .../spark/sql/connect/client/SparkResult.scala | 37 + .../connect/client/arrow/ArrowDeserializer.scala | 1 + .../connect/client/arrow/ArrowEncoderUtils.scala | 2 - .../sql/connect/client/arrow/ArrowSerializer.scala | 1 + .../org/apache/spark/sql/ClientE2ETestSuite.scala | 2 +- .../connect/client/arrow/ArrowEncoderSuite.scala | 1 + 14 files changed, 133 insertions(+), 67 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 0f7b376955c..8a7dce3987a 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2832,7 +2832,7 @@ class Dataset[T] private[sql] ( /** * Returns an iterator that contains all rows in this Dataset. * - * The returned iterator implements [[AutoCloseable]]. For memory management it is better to + * The returned iterator implements [[AutoCloseable]]. For resource management it is better to * close it once you are done. If you don't close it, it and the underlying data will be cleaned * up once the iterator is garbage collected. * @@ -2840,7 +2840,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def toLocalIterator(): java.util.Iterator[T] = { -collectResult().destructiveIterator +collectResult().destructiveIterator.asJava } /** diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 59f3f3526ab..355d7edadc7 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -252,10 +252,8 @@ class SparkSession private[sql] ( .setSql(sqlText) .addAllPosArgs(args.map(toLiteralProto).toIterable.asJava))) val plan = proto.Plan.newBuilder().setCommand(cmd) -val responseSeq = client.execute(plan.build()).asScala.toSeq - -// seq
[spark] branch branch-3.5 updated: [SPARK-44636][CONNECT] Leave no dangling iterators
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new ae1a5c9b86f [SPARK-44636][CONNECT] Leave no dangling iterators ae1a5c9b86f is described below commit ae1a5c9b86f81dfed7471bda6db48f04eb6906ae Author: Alice Sayutina AuthorDate: Wed Aug 2 17:41:47 2023 -0400 [SPARK-44636][CONNECT] Leave no dangling iterators ### What changes were proposed in this pull request? Minorly refactored execute functions to not leave dangling iterators (Note: we also should do that with SparkResult, however in almost all cases there should be no problem with iterators not consumed). ### Why are the changes needed? Needed for ongoing work regarding session reattachment. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? This is intended to be tested after session reattachment is complete (cc juliuszsompolski). Closes #42298 from cdkrot/dangling_iterators. Lead-authored-by: Alice Sayutina Co-authored-by: Alice Sayutina Co-authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell (cherry picked from commit 784f1d0da7f9d96bbc8ab2dda9d9556691012e17) Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 25 +- .../sql/connect/client/SparkConnectClient.scala| 6 ++ .../connect/client/SparkConnectClientSuite.scala | 19 3 files changed, 40 insertions(+), 10 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index a3d82156a03..59f3f3526ab 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -252,10 +252,12 @@ class SparkSession private[sql] ( .setSql(sqlText) .addAllPosArgs(args.map(toLiteralProto).toIterable.asJava))) val plan = proto.Plan.newBuilder().setCommand(cmd) -val responseIter = client.execute(plan.build()) +val responseSeq = client.execute(plan.build()).asScala.toSeq -// Note: .toSeq makes the stream be consumed and closed. -val response = responseIter.asScala.toSeq +// sequence is a lazy stream, force materialize it to make sure it is consumed. +responseSeq.foreach(_ => ()) + +val response = responseSeq .find(_.hasSqlCommandResult) .getOrElse(throw new RuntimeException("SQLCommandResult must be present")) @@ -309,10 +311,12 @@ class SparkSession private[sql] ( .setSql(sqlText) .putAllArgs(args.asScala.mapValues(toLiteralProto).toMap.asJava))) val plan = proto.Plan.newBuilder().setCommand(cmd) - val responseIter = client.execute(plan.build()) + val responseSeq = client.execute(plan.build()).asScala.toSeq + + // sequence is a lazy stream, force materialize it to make sure it is consumed. + responseSeq.foreach(_ => ()) - // Note: .toSeq makes the stream be consumed and closed. - val response = responseIter.asScala.toSeq + val response = responseSeq .find(_.hasSqlCommandResult) .getOrElse(throw new RuntimeException("SQLCommandResult must be present")) @@ -549,14 +553,15 @@ class SparkSession private[sql] ( private[sql] def execute(command: proto.Command): Seq[ExecutePlanResponse] = { val plan = proto.Plan.newBuilder().setCommand(command).build() -client.execute(plan).asScala.toSeq +val seq = client.execute(plan).asScala.toSeq +// sequence is a lazy stream, force materialize it to make sure it is consumed. +seq.foreach(_ => ()) +seq } private[sql] def registerUdf(udf: proto.CommonInlineUserDefinedFunction): Unit = { val command = proto.Command.newBuilder().setRegisterFunction(udf).build() -val plan = proto.Plan.newBuilder().setCommand(command).build() - -client.execute(plan).asScala.foreach(_ => ()) +execute(command) } @DeveloperApi diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index aac5e6b9cc3..3d20be8 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -72,6 +72,12 @@ private[sql] class SparkConnectClient( bstub.analyzePlan(request) } + /** + * Execute the p
[spark] branch master updated: [SPARK-44636][CONNECT] Leave no dangling iterators
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 784f1d0da7f [SPARK-44636][CONNECT] Leave no dangling iterators 784f1d0da7f is described below commit 784f1d0da7f9d96bbc8ab2dda9d9556691012e17 Author: Alice Sayutina AuthorDate: Wed Aug 2 17:41:47 2023 -0400 [SPARK-44636][CONNECT] Leave no dangling iterators ### What changes were proposed in this pull request? Minorly refactored execute functions to not leave dangling iterators (Note: we also should do that with SparkResult, however in almost all cases there should be no problem with iterators not consumed). ### Why are the changes needed? Needed for ongoing work regarding session reattachment. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? This is intended to be tested after session reattachment is complete (cc juliuszsompolski). Closes #42298 from cdkrot/dangling_iterators. Lead-authored-by: Alice Sayutina Co-authored-by: Alice Sayutina Co-authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 25 +- .../sql/connect/client/SparkConnectClient.scala| 6 ++ .../connect/client/SparkConnectClientSuite.scala | 19 3 files changed, 40 insertions(+), 10 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index a3d82156a03..59f3f3526ab 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -252,10 +252,12 @@ class SparkSession private[sql] ( .setSql(sqlText) .addAllPosArgs(args.map(toLiteralProto).toIterable.asJava))) val plan = proto.Plan.newBuilder().setCommand(cmd) -val responseIter = client.execute(plan.build()) +val responseSeq = client.execute(plan.build()).asScala.toSeq -// Note: .toSeq makes the stream be consumed and closed. -val response = responseIter.asScala.toSeq +// sequence is a lazy stream, force materialize it to make sure it is consumed. +responseSeq.foreach(_ => ()) + +val response = responseSeq .find(_.hasSqlCommandResult) .getOrElse(throw new RuntimeException("SQLCommandResult must be present")) @@ -309,10 +311,12 @@ class SparkSession private[sql] ( .setSql(sqlText) .putAllArgs(args.asScala.mapValues(toLiteralProto).toMap.asJava))) val plan = proto.Plan.newBuilder().setCommand(cmd) - val responseIter = client.execute(plan.build()) + val responseSeq = client.execute(plan.build()).asScala.toSeq + + // sequence is a lazy stream, force materialize it to make sure it is consumed. + responseSeq.foreach(_ => ()) - // Note: .toSeq makes the stream be consumed and closed. - val response = responseIter.asScala.toSeq + val response = responseSeq .find(_.hasSqlCommandResult) .getOrElse(throw new RuntimeException("SQLCommandResult must be present")) @@ -549,14 +553,15 @@ class SparkSession private[sql] ( private[sql] def execute(command: proto.Command): Seq[ExecutePlanResponse] = { val plan = proto.Plan.newBuilder().setCommand(command).build() -client.execute(plan).asScala.toSeq +val seq = client.execute(plan).asScala.toSeq +// sequence is a lazy stream, force materialize it to make sure it is consumed. +seq.foreach(_ => ()) +seq } private[sql] def registerUdf(udf: proto.CommonInlineUserDefinedFunction): Unit = { val command = proto.Command.newBuilder().setRegisterFunction(udf).build() -val plan = proto.Plan.newBuilder().setCommand(command).build() - -client.execute(plan).asScala.foreach(_ => ()) +execute(command) } @DeveloperApi diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index aac5e6b9cc3..3d20be8 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -72,6 +72,12 @@ private[sql] class SparkConnectClient( bstub.analyzePlan(request) } + /** + * Execute the plan and return response iterator. + * + * It returns an open iterator. The caller needs to ensure that this iterator is
[spark] branch branch-3.5 updated: [SPARK-44421][FOLLOWUP] Minor rename of ResponseComplete to ResultComplete
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new fde43c94ee4 [SPARK-44421][FOLLOWUP] Minor rename of ResponseComplete to ResultComplete fde43c94ee4 is described below commit fde43c94ee4e3bf9e95166de7387c0ec925984e6 Author: Juliusz Sompolski AuthorDate: Wed Aug 2 15:18:08 2023 -0400 [SPARK-44421][FOLLOWUP] Minor rename of ResponseComplete to ResultComplete ### What changes were proposed in this pull request? In an earlier draft, the message was ResponseComplete. Missed renaming it in a few places. ### Why are the changes needed? Minor followup cleanup. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? No code semantics changes, just rename. Closes #42300 from juliuszsompolski/SPARK-44421-followup. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell (cherry picked from commit 6882482fcdfb438c3b5a0b1b2ece58658ac1f0e7) Signed-off-by: Herman van Hovell --- .../client/ExecutePlanResponseReattachableIterator.scala | 14 +++--- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala index 00787b8f94d..008b3c3dd5c 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala @@ -85,10 +85,10 @@ class ExecutePlanResponseReattachableIterator( // ResponseId of the last response returned by next() private var lastReturnedResponseId: Option[String] = None - // True after ResponseComplete message was seen in the stream. + // True after ResultComplete message was seen in the stream. // Server will always send this message at the end of the stream, if the underlying iterator // finishes without producing one, another iterator needs to be reattached. - private var responseComplete: Boolean = false + private var resultComplete: Boolean = false // Initial iterator comes from ExecutePlan request. // Note: This is not retried, because no error would ever be thrown here, and GRPC will only @@ -97,7 +97,7 @@ class ExecutePlanResponseReattachableIterator( rawBlockingStub.executePlan(initialRequest) override def next(): proto.ExecutePlanResponse = synchronized { -// hasNext will trigger reattach in case the stream completed without responseComplete +// hasNext will trigger reattach in case the stream completed without resultComplete if (!hasNext()) { throw new java.util.NoSuchElementException() } @@ -118,7 +118,7 @@ class ExecutePlanResponseReattachableIterator( // Record last returned response, to know where to restart in case of reattach. lastReturnedResponseId = Some(ret.getResponseId) if (ret.hasResultComplete) { - responseComplete = true + resultComplete = true releaseExecute(None) // release all } else { releaseExecute(lastReturnedResponseId) // release until this response @@ -127,7 +127,7 @@ class ExecutePlanResponseReattachableIterator( } override def hasNext(): Boolean = synchronized { -if (responseComplete) { +if (resultComplete) { // After response complete response return false } @@ -144,10 +144,10 @@ class ExecutePlanResponseReattachableIterator( // Graceful reattach: // If iterator ended, but there was no ResultComplete, it means that there is more, // and we need to reattach. - if (!hasNext && !responseComplete) { + if (!hasNext && !resultComplete) { do { iterator = rawBlockingStub.reattachExecute(createReattachExecuteRequest()) - assert(!responseComplete) // shouldn't change... + assert(!resultComplete) // shouldn't change... hasNext = iterator.hasNext() // It's possible that the new iterator will be empty, so we need to loop to get another. // Eventually, there will be a non empty iterator, because there's always a ResultComplete - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44421][FOLLOWUP] Minor rename of ResponseComplete to ResultComplete
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6882482fcdf [SPARK-44421][FOLLOWUP] Minor rename of ResponseComplete to ResultComplete 6882482fcdf is described below commit 6882482fcdfb438c3b5a0b1b2ece58658ac1f0e7 Author: Juliusz Sompolski AuthorDate: Wed Aug 2 15:18:08 2023 -0400 [SPARK-44421][FOLLOWUP] Minor rename of ResponseComplete to ResultComplete ### What changes were proposed in this pull request? In an earlier draft, the message was ResponseComplete. Missed renaming it in a few places. ### Why are the changes needed? Minor followup cleanup. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? No code semantics changes, just rename. Closes #42300 from juliuszsompolski/SPARK-44421-followup. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell --- .../client/ExecutePlanResponseReattachableIterator.scala | 14 +++--- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala index 00787b8f94d..008b3c3dd5c 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala @@ -85,10 +85,10 @@ class ExecutePlanResponseReattachableIterator( // ResponseId of the last response returned by next() private var lastReturnedResponseId: Option[String] = None - // True after ResponseComplete message was seen in the stream. + // True after ResultComplete message was seen in the stream. // Server will always send this message at the end of the stream, if the underlying iterator // finishes without producing one, another iterator needs to be reattached. - private var responseComplete: Boolean = false + private var resultComplete: Boolean = false // Initial iterator comes from ExecutePlan request. // Note: This is not retried, because no error would ever be thrown here, and GRPC will only @@ -97,7 +97,7 @@ class ExecutePlanResponseReattachableIterator( rawBlockingStub.executePlan(initialRequest) override def next(): proto.ExecutePlanResponse = synchronized { -// hasNext will trigger reattach in case the stream completed without responseComplete +// hasNext will trigger reattach in case the stream completed without resultComplete if (!hasNext()) { throw new java.util.NoSuchElementException() } @@ -118,7 +118,7 @@ class ExecutePlanResponseReattachableIterator( // Record last returned response, to know where to restart in case of reattach. lastReturnedResponseId = Some(ret.getResponseId) if (ret.hasResultComplete) { - responseComplete = true + resultComplete = true releaseExecute(None) // release all } else { releaseExecute(lastReturnedResponseId) // release until this response @@ -127,7 +127,7 @@ class ExecutePlanResponseReattachableIterator( } override def hasNext(): Boolean = synchronized { -if (responseComplete) { +if (resultComplete) { // After response complete response return false } @@ -144,10 +144,10 @@ class ExecutePlanResponseReattachableIterator( // Graceful reattach: // If iterator ended, but there was no ResultComplete, it means that there is more, // and we need to reattach. - if (!hasNext && !responseComplete) { + if (!hasNext && !resultComplete) { do { iterator = rawBlockingStub.reattachExecute(createReattachExecuteRequest()) - assert(!responseComplete) // shouldn't change... + assert(!resultComplete) // shouldn't change... hasNext = iterator.hasNext() // It's possible that the new iterator will be empty, so we need to loop to get another. // Eventually, there will be a non empty iterator, because there's always a ResultComplete - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.5 updated: [SPARK-44637][CONNECT] Synchronize accesses to ExecuteResponseObserver
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 1d43683c5f0 [SPARK-44637][CONNECT] Synchronize accesses to ExecuteResponseObserver 1d43683c5f0 is described below commit 1d43683c5f0ad1aed25cfd9d4361fed866b3d1af Author: Juliusz Sompolski AuthorDate: Wed Aug 2 15:07:12 2023 -0400 [SPARK-44637][CONNECT] Synchronize accesses to ExecuteResponseObserver ### What changes were proposed in this pull request? Function `removeResponsesUntilId` is used by `ReleaseExecute` RPC, and that needs to be synchronized against `removeCachedResponses` running from `consumeResponse` for `ExecutePlan` or `ReattachExecute` RPC. In general, all public accesses to ExecuteResponseObserver should best be synchronized. ### Why are the changes needed? Fix synchronization bug caught by testing of python client. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Caught in https://github.com/apache/spark/pull/42235, but want to fix separately because this is a server side change. Closes #42299 from juliuszsompolski/SPARK-44637. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell (cherry picked from commit 26c7e55f19993ef265b8730503c1ffa4ee697347) Signed-off-by: Herman van Hovell --- .../apache/spark/sql/connect/execution/ExecuteResponseObserver.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala index 5966e6cf0fc..8af0f72b8da 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala @@ -179,7 +179,7 @@ private[connect] class ExecuteResponseObserver[T <: MessageLite](val executeHold } /** Get the index in the stream for given response id. */ - def getResponseIndexById(responseId: String): Long = { + def getResponseIndexById(responseId: String): Long = synchronized { responseIdToIndex.getOrElse( responseId, throw new SparkSQLException( @@ -188,7 +188,7 @@ private[connect] class ExecuteResponseObserver[T <: MessageLite](val executeHold } /** Remove cached responses up to and including response with given id. */ - def removeResponsesUntilId(responseId: String): Unit = { + def removeResponsesUntilId(responseId: String): Unit = synchronized { val index = getResponseIndexById(responseId) removeResponsesUntilIndex(index) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44637][CONNECT] Synchronize accesses to ExecuteResponseObserver
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 26c7e55f199 [SPARK-44637][CONNECT] Synchronize accesses to ExecuteResponseObserver 26c7e55f199 is described below commit 26c7e55f19993ef265b8730503c1ffa4ee697347 Author: Juliusz Sompolski AuthorDate: Wed Aug 2 15:07:12 2023 -0400 [SPARK-44637][CONNECT] Synchronize accesses to ExecuteResponseObserver ### What changes were proposed in this pull request? Function `removeResponsesUntilId` is used by `ReleaseExecute` RPC, and that needs to be synchronized against `removeCachedResponses` running from `consumeResponse` for `ExecutePlan` or `ReattachExecute` RPC. In general, all public accesses to ExecuteResponseObserver should best be synchronized. ### Why are the changes needed? Fix synchronization bug caught by testing of python client. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Caught in https://github.com/apache/spark/pull/42235, but want to fix separately because this is a server side change. Closes #42299 from juliuszsompolski/SPARK-44637. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell --- .../apache/spark/sql/connect/execution/ExecuteResponseObserver.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala index 5966e6cf0fc..8af0f72b8da 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala @@ -179,7 +179,7 @@ private[connect] class ExecuteResponseObserver[T <: MessageLite](val executeHold } /** Get the index in the stream for given response id. */ - def getResponseIndexById(responseId: String): Long = { + def getResponseIndexById(responseId: String): Long = synchronized { responseIdToIndex.getOrElse( responseId, throw new SparkSQLException( @@ -188,7 +188,7 @@ private[connect] class ExecuteResponseObserver[T <: MessageLite](val executeHold } /** Remove cached responses up to and including response with given id. */ - def removeResponsesUntilId(responseId: String): Unit = { + def removeResponsesUntilId(responseId: String): Unit = synchronized { val index = getResponseIndexById(responseId) removeResponsesUntilIndex(index) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.5 updated: [SPARK-29497][CONNECT] Throw error when UDF is not deserializable
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 8537fa634cd [SPARK-29497][CONNECT] Throw error when UDF is not deserializable 8537fa634cd is described below commit 8537fa634cd02f46e7b42afd6b35f877f3a2c161 Author: Herman van Hovell AuthorDate: Tue Aug 1 14:53:54 2023 -0400 [SPARK-29497][CONNECT] Throw error when UDF is not deserializable ### What changes were proposed in this pull request? This PR adds a better error message when a JVM UDF cannot be deserialized. ### Why are the changes needed? In some cases a UDF cannot be deserialized. The happens when a lambda references itself (typically through the capturing class). Java cannot deserialize such an object graph because SerializedLambda's are serialization proxies which need the full graph to be deserialized before they can be transformed into the actual lambda. This is not possible if there is such a cycle. This PR adds a more readable and understandable error when this happens, the original java one is a `ClassCastExcep [...] ### Does this PR introduce _any_ user-facing change? Yes. It will throw an error on the client when a UDF is not deserializable. The error is better and more actionable then what we got before. ### How was this patch tested? Added tests. Closes #42245 from hvanhovell/SPARK-29497. Lead-authored-by: Herman van Hovell Co-authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit f54b402021785e0b0ec976ec889de67d3b2fdc6e) Signed-off-by: Herman van Hovell --- .../org/apache/spark/util/SparkSerDeUtils.scala| 21 ++- .../sql/expressions/UserDefinedFunction.scala | 24 +++- .../spark/sql/UserDefinedFunctionSuite.scala | 44 -- .../main/scala/org/apache/spark/util/Utils.scala | 23 +-- 4 files changed, 85 insertions(+), 27 deletions(-) diff --git a/common/utils/src/main/scala/org/apache/spark/util/SparkSerDeUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/SparkSerDeUtils.scala index 3069e4c36a7..9b6174c47bd 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/SparkSerDeUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/SparkSerDeUtils.scala @@ -16,9 +16,9 @@ */ package org.apache.spark.util -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream, ObjectStreamClass} -object SparkSerDeUtils { +trait SparkSerDeUtils { /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -34,4 +34,21 @@ object SparkSerDeUtils { val ois = new ObjectInputStream(bis) ois.readObject.asInstanceOf[T] } + + /** + * Deserialize an object using Java serialization and the given ClassLoader + */ + def deserialize[T](bytes: Array[Byte], loader: ClassLoader): T = { +val bis = new ByteArrayInputStream(bytes) +val ois = new ObjectInputStream(bis) { + override def resolveClass(desc: ObjectStreamClass): Class[_] = { +// scalastyle:off classforname +Class.forName(desc.getName, false, loader) +// scalastyle:on classforname + } +} +ois.readObject.asInstanceOf[T] + } } + +object SparkSerDeUtils extends SparkSerDeUtils diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 3a38029c265..e060dba0b7e 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -18,16 +18,18 @@ package org.apache.spark.sql.expressions import scala.collection.JavaConverters._ import scala.reflect.runtime.universe.TypeTag +import scala.util.control.NonFatal import com.google.protobuf.ByteString +import org.apache.spark.SparkException import org.apache.spark.connect.proto import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, UdfPacket} import org.apache.spark.sql.types.DataType -import org.apache.spark.util.SparkSerDeUtils +import org.apache.spark.util.{SparkClassUtils, SparkSerDeUtils} /** * A user-defined function. To create one, use the `udf` functions in `functions`. @@ -144,6 +146,25 @@ case
[spark] branch master updated (4f62f8a718e -> f54b4020217)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 4f62f8a718e [SPARK-44613][CONNECT] Add Encoders object add f54b4020217 [SPARK-29497][CONNECT] Throw error when UDF is not deserializable No new revisions were added by this update. Summary of changes: .../org/apache/spark/util/SparkSerDeUtils.scala| 21 ++- .../sql/expressions/UserDefinedFunction.scala | 24 +++- .../spark/sql/UserDefinedFunctionSuite.scala | 44 -- .../main/scala/org/apache/spark/util/Utils.scala | 23 +-- 4 files changed, 85 insertions(+), 27 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.5 updated: [SPARK-44613][CONNECT] Add Encoders object
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new bde7aa61ce3 [SPARK-44613][CONNECT] Add Encoders object bde7aa61ce3 is described below commit bde7aa61ce3de15323a8920e8114a681fcd17000 Author: Herman van Hovell AuthorDate: Tue Aug 1 14:39:38 2023 -0400 [SPARK-44613][CONNECT] Add Encoders object ### What changes were proposed in this pull request? This PR adds the org.apache.spark.sql.Encoders object to Connect. ### Why are the changes needed? To increase compatibility with the SQL Dataframe API ### Does this PR introduce _any_ user-facing change? Yes, it adds missing functionality. ### How was this patch tested? Added a couple of java based tests. Closes #42264 from hvanhovell/SPARK-44613. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 4f62f8a718e80dca13a1d44b6fdf8857f037c15e) Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Encoders.scala | 262 + .../spark/sql/connect/client/SparkResult.scala | 14 +- .../org/apache/spark/sql/JavaEncoderSuite.java | 94 .../CheckConnectJvmClientCompatibility.scala | 8 +- .../connect/client/util/RemoteSparkSession.scala | 2 +- 5 files changed, 371 insertions(+), 9 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala new file mode 100644 index 000..3f2f7ec96d4 --- /dev/null +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -0,0 +1,262 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ + +/** + * Methods for creating an [[Encoder]]. + * + * @since 3.5.0 + */ +object Encoders { + + /** + * An encoder for nullable boolean type. The Scala primitive encoder is available as + * [[scalaBoolean]]. + * @since 3.5.0 + */ + def BOOLEAN: Encoder[java.lang.Boolean] = BoxedBooleanEncoder + + /** + * An encoder for nullable byte type. The Scala primitive encoder is available as [[scalaByte]]. + * @since 3.5.0 + */ + def BYTE: Encoder[java.lang.Byte] = BoxedByteEncoder + + /** + * An encoder for nullable short type. The Scala primitive encoder is available as + * [[scalaShort]]. + * @since 3.5.0 + */ + def SHORT: Encoder[java.lang.Short] = BoxedShortEncoder + + /** + * An encoder for nullable int type. The Scala primitive encoder is available as [[scalaInt]]. + * @since 3.5.0 + */ + def INT: Encoder[java.lang.Integer] = BoxedIntEncoder + + /** + * An encoder for nullable long type. The Scala primitive encoder is available as [[scalaLong]]. + * @since 3.5.0 + */ + def LONG: Encoder[java.lang.Long] = BoxedLongEncoder + + /** + * An encoder for nullable float type. The Scala primitive encoder is available as + * [[scalaFloat]]. + * @since 3.5.0 + */ + def FLOAT: Encoder[java.lang.Float] = BoxedFloatEncoder + + /** + * An encoder for nullable double type. The Scala primitive encoder is available as + * [[scalaDouble]]. + * @since 3.5.0 + */ + def DOUBLE: Encoder[java.lang.Double] = BoxedDoubleEncoder + + /** + * An encoder for nullable string type. + * + * @since 3.5.0 + */ + def STRING: Encoder[java.lang.String] = StringEncoder + + /** + * An encoder for nullable decimal type. + * + * @since 3.5.0 + */ + def DECIMAL: Encoder[java.math.BigDecimal] = DEFAULT_JAVA_DECIMAL_ENCODER + + /** + * An encoder for nullable date type. + * + * @since 3.5.0 + */ + def DATE: Encoder[java.sql.Date] = DateEncoder(lenientSerialization = false) + +
[spark] branch master updated: [SPARK-44613][CONNECT] Add Encoders object
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 4f62f8a718e [SPARK-44613][CONNECT] Add Encoders object 4f62f8a718e is described below commit 4f62f8a718e80dca13a1d44b6fdf8857f037c15e Author: Herman van Hovell AuthorDate: Tue Aug 1 14:39:38 2023 -0400 [SPARK-44613][CONNECT] Add Encoders object ### What changes were proposed in this pull request? This PR adds the org.apache.spark.sql.Encoders object to Connect. ### Why are the changes needed? To increase compatibility with the SQL Dataframe API ### Does this PR introduce _any_ user-facing change? Yes, it adds missing functionality. ### How was this patch tested? Added a couple of java based tests. Closes #42264 from hvanhovell/SPARK-44613. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Encoders.scala | 262 + .../spark/sql/connect/client/SparkResult.scala | 14 +- .../org/apache/spark/sql/JavaEncoderSuite.java | 94 .../CheckConnectJvmClientCompatibility.scala | 8 +- .../connect/client/util/RemoteSparkSession.scala | 2 +- 5 files changed, 371 insertions(+), 9 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala new file mode 100644 index 000..3f2f7ec96d4 --- /dev/null +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -0,0 +1,262 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ + +/** + * Methods for creating an [[Encoder]]. + * + * @since 3.5.0 + */ +object Encoders { + + /** + * An encoder for nullable boolean type. The Scala primitive encoder is available as + * [[scalaBoolean]]. + * @since 3.5.0 + */ + def BOOLEAN: Encoder[java.lang.Boolean] = BoxedBooleanEncoder + + /** + * An encoder for nullable byte type. The Scala primitive encoder is available as [[scalaByte]]. + * @since 3.5.0 + */ + def BYTE: Encoder[java.lang.Byte] = BoxedByteEncoder + + /** + * An encoder for nullable short type. The Scala primitive encoder is available as + * [[scalaShort]]. + * @since 3.5.0 + */ + def SHORT: Encoder[java.lang.Short] = BoxedShortEncoder + + /** + * An encoder for nullable int type. The Scala primitive encoder is available as [[scalaInt]]. + * @since 3.5.0 + */ + def INT: Encoder[java.lang.Integer] = BoxedIntEncoder + + /** + * An encoder for nullable long type. The Scala primitive encoder is available as [[scalaLong]]. + * @since 3.5.0 + */ + def LONG: Encoder[java.lang.Long] = BoxedLongEncoder + + /** + * An encoder for nullable float type. The Scala primitive encoder is available as + * [[scalaFloat]]. + * @since 3.5.0 + */ + def FLOAT: Encoder[java.lang.Float] = BoxedFloatEncoder + + /** + * An encoder for nullable double type. The Scala primitive encoder is available as + * [[scalaDouble]]. + * @since 3.5.0 + */ + def DOUBLE: Encoder[java.lang.Double] = BoxedDoubleEncoder + + /** + * An encoder for nullable string type. + * + * @since 3.5.0 + */ + def STRING: Encoder[java.lang.String] = StringEncoder + + /** + * An encoder for nullable decimal type. + * + * @since 3.5.0 + */ + def DECIMAL: Encoder[java.math.BigDecimal] = DEFAULT_JAVA_DECIMAL_ENCODER + + /** + * An encoder for nullable date type. + * + * @since 3.5.0 + */ + def DATE: Encoder[java.sql.Date] = DateEncoder(lenientSerialization = false) + + /** + * Creates an encoder that serializes instances of the `java.time.LocalDate` class to the + * internal repres
[spark] branch branch-3.5 updated: [SPARK-44311][CONNECT][SQL] Improved support for UDFs on value classes
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 0843b7741fa [SPARK-44311][CONNECT][SQL] Improved support for UDFs on value classes 0843b7741fa is described below commit 0843b7741fa959173fcc66067eedda9be501192c Author: Emil Ejbyfeldt AuthorDate: Tue Aug 1 10:50:04 2023 -0400 [SPARK-44311][CONNECT][SQL] Improved support for UDFs on value classes ### What changes were proposed in this pull request? This pr fixes using UDFs on value classes when it serialized as in underlying type. Previously it would only work if one either defined a UDF taking the underlying type and/or for cases where the schema derived does not "unbox" the value to its underlying type. Before this change the following code: ``` final case class ValueClass(a: Int) extends AnyVal final case class Wrapper(v: ValueClass) val f = udf((a: ValueClass) => a.a > 0) spark.createDataset(Seq(Wrapper(ValueClass(1.filter(f(col("v"))).show() ``` would fails with ``` java.lang.ClassCastException: class org.apache.spark.sql.types.IntegerType$ cannot be cast to class org.apache.spark.sql.types.StructType (org.apache.spark.sql.types.IntegerType$ and org.apache.spark.sql.types.StructType are in unnamed module of loader 'app') at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveEncodersInUDF$$anonfun$apply$42$$anonfun$applyOrElse$218.$anonfun$applyOrElse$220(Analyzer.scala:3241) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveEncodersInUDF$$anonfun$apply$42$$anonfun$applyOrElse$218.$anonfun$applyOrElse$219(Analyzer.scala:3239) at scala.collection.immutable.List.map(List.scala:246) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveEncodersInUDF$$anonfun$apply$42$$anonfun$applyOrElse$218.applyOrElse(Analyzer.scala:3237) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveEncodersInUDF$$anonfun$apply$42$$anonfun$applyOrElse$218.applyOrElse(Analyzer.scala:3234) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUpWithPruning$2(TreeNode.scala:566) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:104) at org.apache.spark.sql.catalyst.trees.TreeNode.transformUpWithPruning(TreeNode.scala:566) ``` ### Why are the changes needed? This is something as a user I would expect to just work. ### Does this PR introduce _any_ user-facing change? Yes, it if fixes using a UDF on value class that is serialized as it underlying type. ### How was this patch tested? Existing test and new tests cases in DatasetSuite.scala Closes #41876 from eejbyfeldt/SPARK-44311. Authored-by: Emil Ejbyfeldt Signed-off-by: Herman van Hovell (cherry picked from commit 821026bc730ce87e6e97d304c7673bfcb23fd03a) Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/analysis/Analyzer.scala | 7 ++- .../spark/sql/catalyst/expressions/ScalaUDF.scala | 4 +++- .../scala/org/apache/spark/sql/DatasetSuite.scala | 24 ++ 3 files changed, 33 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 30c6e4b4bc0..7f2471c9e19 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -3245,7 +3245,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val dataType = udf.children(i).dataType encOpt.map { enc => val attrs = if (enc.isSerializedAsStructForTopLevel) { -DataTypeUtils.toAttributes(dataType.asInstanceOf[StructType]) +// Value class that has been replaced with its underlying type +if (enc.schema.fields.size == 1 && enc.schema.fields.head.dataType == dataType) { + DataTypeUtils.toAttributes(enc.schema.asInstanceOf[StructType]) +} else { + DataTypeUtils.toAttributes(dataType.asInstanceOf[StructType]) +} } else { // the field name doesn't matter here, so we use // a simple literal to avoid any overhead diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 402
[spark] branch master updated: [SPARK-44311][CONNECT][SQL] Improved support for UDFs on value classes
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 821026bc730 [SPARK-44311][CONNECT][SQL] Improved support for UDFs on value classes 821026bc730 is described below commit 821026bc730ce87e6e97d304c7673bfcb23fd03a Author: Emil Ejbyfeldt AuthorDate: Tue Aug 1 10:50:04 2023 -0400 [SPARK-44311][CONNECT][SQL] Improved support for UDFs on value classes ### What changes were proposed in this pull request? This pr fixes using UDFs on value classes when it serialized as in underlying type. Previously it would only work if one either defined a UDF taking the underlying type and/or for cases where the schema derived does not "unbox" the value to its underlying type. Before this change the following code: ``` final case class ValueClass(a: Int) extends AnyVal final case class Wrapper(v: ValueClass) val f = udf((a: ValueClass) => a.a > 0) spark.createDataset(Seq(Wrapper(ValueClass(1.filter(f(col("v"))).show() ``` would fails with ``` java.lang.ClassCastException: class org.apache.spark.sql.types.IntegerType$ cannot be cast to class org.apache.spark.sql.types.StructType (org.apache.spark.sql.types.IntegerType$ and org.apache.spark.sql.types.StructType are in unnamed module of loader 'app') at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveEncodersInUDF$$anonfun$apply$42$$anonfun$applyOrElse$218.$anonfun$applyOrElse$220(Analyzer.scala:3241) at scala.Option.map(Option.scala:242) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveEncodersInUDF$$anonfun$apply$42$$anonfun$applyOrElse$218.$anonfun$applyOrElse$219(Analyzer.scala:3239) at scala.collection.immutable.List.map(List.scala:246) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveEncodersInUDF$$anonfun$apply$42$$anonfun$applyOrElse$218.applyOrElse(Analyzer.scala:3237) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveEncodersInUDF$$anonfun$apply$42$$anonfun$applyOrElse$218.applyOrElse(Analyzer.scala:3234) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUpWithPruning$2(TreeNode.scala:566) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:104) at org.apache.spark.sql.catalyst.trees.TreeNode.transformUpWithPruning(TreeNode.scala:566) ``` ### Why are the changes needed? This is something as a user I would expect to just work. ### Does this PR introduce _any_ user-facing change? Yes, it if fixes using a UDF on value class that is serialized as it underlying type. ### How was this patch tested? Existing test and new tests cases in DatasetSuite.scala Closes #41876 from eejbyfeldt/SPARK-44311. Authored-by: Emil Ejbyfeldt Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/analysis/Analyzer.scala | 7 ++- .../spark/sql/catalyst/expressions/ScalaUDF.scala | 4 +++- .../scala/org/apache/spark/sql/DatasetSuite.scala | 24 ++ 3 files changed, 33 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 92e550ea941..dc42d27e8e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -3251,7 +3251,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val dataType = udf.children(i).dataType encOpt.map { enc => val attrs = if (enc.isSerializedAsStructForTopLevel) { -DataTypeUtils.toAttributes(dataType.asInstanceOf[StructType]) +// Value class that has been replaced with its underlying type +if (enc.schema.fields.size == 1 && enc.schema.fields.head.dataType == dataType) { + DataTypeUtils.toAttributes(enc.schema.asInstanceOf[StructType]) +} else { + DataTypeUtils.toAttributes(dataType.asInstanceOf[StructType]) +} } else { // the field name doesn't matter here, so we use // a simple literal to avoid any overhead diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 40274a83340..910960bf84b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF
[spark] branch branch-3.5 updated: [SPARK-44611][CONNECT] Do not exclude scala-xml
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 042e099bf0a [SPARK-44611][CONNECT] Do not exclude scala-xml 042e099bf0a is described below commit 042e099bf0adb0bc2f5f980ba82c7c5fef363d02 Author: Herman van Hovell AuthorDate: Mon Jul 31 21:01:51 2023 -0400 [SPARK-44611][CONNECT] Do not exclude scala-xml ### What changes were proposed in this pull request? This PR removes the scala-xml exclusion for ammonite. ### Why are the changes needed? Ammonite does not work well without scala-xml. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually Closes #42246 from hvanhovell/SPARK-44611. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 9a2abe42cf484fb51410e0e3b86776d6e673d8b6) Signed-off-by: Herman van Hovell --- connector/connect/client/jvm/pom.xml | 6 -- 1 file changed, 6 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 503888ce095..f327db7c193 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -94,12 +94,6 @@ ammonite_${scala.version} ${ammonite.version} provided - - - org.scala-lang.modules - scala-xml_${scala.binary.version} - - org.apache.spark - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44611][CONNECT] Do not exclude scala-xml
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9a2abe42cf4 [SPARK-44611][CONNECT] Do not exclude scala-xml 9a2abe42cf4 is described below commit 9a2abe42cf484fb51410e0e3b86776d6e673d8b6 Author: Herman van Hovell AuthorDate: Mon Jul 31 21:01:51 2023 -0400 [SPARK-44611][CONNECT] Do not exclude scala-xml ### What changes were proposed in this pull request? This PR removes the scala-xml exclusion for ammonite. ### Why are the changes needed? Ammonite does not work well without scala-xml. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually Closes #42246 from hvanhovell/SPARK-44611. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- connector/connect/client/jvm/pom.xml | 6 -- 1 file changed, 6 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index b85201f49a2..04eed736fba 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -94,12 +94,6 @@ ammonite_${scala.version} ${ammonite.version} provided - - - org.scala-lang.modules - scala-xml_${scala.binary.version} - - org.apache.spark - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.5 updated: [SPARK-43997][CONNECT] Add support for Java UDFs
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new fab48fee86d [SPARK-43997][CONNECT] Add support for Java UDFs fab48fee86d is described below commit fab48fee86da6b6d367adf34154a905482dd7d0d Author: vicennial AuthorDate: Mon Jul 31 12:59:18 2023 -0400 [SPARK-43997][CONNECT] Add support for Java UDFs ### What changes were proposed in this pull request? Adds support for a user to define, register and use Java UDFs. The following APIs are included in this PR: - `def udf(f: UDF4[_, _, _, _, _], returnType: DataType): UserDefinedFunction` for [0-10] arguments. - `def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit` for [0-22] arguments. ### Why are the changes needed? To reach parity with classic Spark. ### Does this PR introduce _any_ user-facing change? Yes. A user can now define, register and use Java UDFs. Refer to the following examples (demonstrated in scala for simplicity) ```scala import org.apache.spark.sql.api.java._ import org.apache.spark.sql.types.LongType spark.udf.register("javaUdf", new UDF1[Long, Long] { override def call(num: Long): Long = num * num * num + 250L }, LongType) spark.sql("select javaUdf(id) from range(5)").as[Long].collect() ``` The output: ```scala Array[Long] = Array(250L, 251L, 258L, 277L, 314L) ``` ### How was this patch tested? E2E tests in `ReplE2ESuite` Closes #42225 from vicennial/javaUdfApi. Authored-by: vicennial Signed-off-by: Herman van Hovell (cherry picked from commit 24f9b26cb1081608ccf58e0eb57e8de278d4f66f) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/UDFRegistration.scala | 266 .../sql/expressions/UserDefinedFunction.scala | 10 +- .../scala/org/apache/spark/sql/functions.scala | 151 +++ .../spark/sql/application/ReplE2ESuite.scala | 30 ++ .../apache/spark/sql/connect/common/UdfUtils.scala | 447 + dev/checkstyle-suppressions.xml| 2 + project/MimaExcludes.scala | 3 + .../java/org/apache/spark/sql/api/java/UDF0.java | 0 .../java/org/apache/spark/sql/api/java/UDF1.java | 0 .../java/org/apache/spark/sql/api/java/UDF10.java | 0 .../java/org/apache/spark/sql/api/java/UDF11.java | 0 .../java/org/apache/spark/sql/api/java/UDF12.java | 0 .../java/org/apache/spark/sql/api/java/UDF13.java | 0 .../java/org/apache/spark/sql/api/java/UDF14.java | 0 .../java/org/apache/spark/sql/api/java/UDF15.java | 0 .../java/org/apache/spark/sql/api/java/UDF16.java | 0 .../java/org/apache/spark/sql/api/java/UDF17.java | 0 .../java/org/apache/spark/sql/api/java/UDF18.java | 0 .../java/org/apache/spark/sql/api/java/UDF19.java | 0 .../java/org/apache/spark/sql/api/java/UDF2.java | 0 .../java/org/apache/spark/sql/api/java/UDF20.java | 0 .../java/org/apache/spark/sql/api/java/UDF21.java | 0 .../java/org/apache/spark/sql/api/java/UDF22.java | 0 .../java/org/apache/spark/sql/api/java/UDF3.java | 0 .../java/org/apache/spark/sql/api/java/UDF4.java | 0 .../java/org/apache/spark/sql/api/java/UDF5.java | 0 .../java/org/apache/spark/sql/api/java/UDF6.java | 0 .../java/org/apache/spark/sql/api/java/UDF7.java | 0 .../java/org/apache/spark/sql/api/java/UDF8.java | 0 .../java/org/apache/spark/sql/api/java/UDF9.java | 0 .../apache/spark/sql/api/java/package-info.java| 0 .../spark/sql/catalyst/encoders/RowEncoder.scala | 2 +- 32 files changed, 909 insertions(+), 2 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 426709b8f18..2e8211a0966 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -20,7 +20,10 @@ package org.apache.spark.sql import scala.reflect.runtime.universe.{typeTag, TypeTag} import org.apache.spark.internal.Logging +import org.apache.spark.sql.api.java._ +import org.apache.spark.sql.connect.common.UdfUtils import org.apache.spark.sql.expressions.{ScalarUserDefinedFunction, UserDefinedFunction} +import org.apache.spark.sql.types.DataType /** * Functions for registering user-defined functions. Use `SparkSession.udf` to access this: @@ -1024,5 +1027,268 @@ class UDFRegistration(session: SparkSession) extends Logging { typeTag[A22]) register(name, udf) } + + // (0 to 22).foreach { i => +
[spark] branch master updated: [SPARK-43997][CONNECT] Add support for Java UDFs
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 24f9b26cb10 [SPARK-43997][CONNECT] Add support for Java UDFs 24f9b26cb10 is described below commit 24f9b26cb1081608ccf58e0eb57e8de278d4f66f Author: vicennial AuthorDate: Mon Jul 31 12:59:18 2023 -0400 [SPARK-43997][CONNECT] Add support for Java UDFs ### What changes were proposed in this pull request? Adds support for a user to define, register and use Java UDFs. The following APIs are included in this PR: - `def udf(f: UDF4[_, _, _, _, _], returnType: DataType): UserDefinedFunction` for [0-10] arguments. - `def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit` for [0-22] arguments. ### Why are the changes needed? To reach parity with classic Spark. ### Does this PR introduce _any_ user-facing change? Yes. A user can now define, register and use Java UDFs. Refer to the following examples (demonstrated in scala for simplicity) ```scala import org.apache.spark.sql.api.java._ import org.apache.spark.sql.types.LongType spark.udf.register("javaUdf", new UDF1[Long, Long] { override def call(num: Long): Long = num * num * num + 250L }, LongType) spark.sql("select javaUdf(id) from range(5)").as[Long].collect() ``` The output: ```scala Array[Long] = Array(250L, 251L, 258L, 277L, 314L) ``` ### How was this patch tested? E2E tests in `ReplE2ESuite` Closes #42225 from vicennial/javaUdfApi. Authored-by: vicennial Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/UDFRegistration.scala | 266 .../sql/expressions/UserDefinedFunction.scala | 10 +- .../scala/org/apache/spark/sql/functions.scala | 151 +++ .../spark/sql/application/ReplE2ESuite.scala | 30 ++ .../apache/spark/sql/connect/common/UdfUtils.scala | 447 + dev/checkstyle-suppressions.xml| 2 + project/MimaExcludes.scala | 3 + .../java/org/apache/spark/sql/api/java/UDF0.java | 0 .../java/org/apache/spark/sql/api/java/UDF1.java | 0 .../java/org/apache/spark/sql/api/java/UDF10.java | 0 .../java/org/apache/spark/sql/api/java/UDF11.java | 0 .../java/org/apache/spark/sql/api/java/UDF12.java | 0 .../java/org/apache/spark/sql/api/java/UDF13.java | 0 .../java/org/apache/spark/sql/api/java/UDF14.java | 0 .../java/org/apache/spark/sql/api/java/UDF15.java | 0 .../java/org/apache/spark/sql/api/java/UDF16.java | 0 .../java/org/apache/spark/sql/api/java/UDF17.java | 0 .../java/org/apache/spark/sql/api/java/UDF18.java | 0 .../java/org/apache/spark/sql/api/java/UDF19.java | 0 .../java/org/apache/spark/sql/api/java/UDF2.java | 0 .../java/org/apache/spark/sql/api/java/UDF20.java | 0 .../java/org/apache/spark/sql/api/java/UDF21.java | 0 .../java/org/apache/spark/sql/api/java/UDF22.java | 0 .../java/org/apache/spark/sql/api/java/UDF3.java | 0 .../java/org/apache/spark/sql/api/java/UDF4.java | 0 .../java/org/apache/spark/sql/api/java/UDF5.java | 0 .../java/org/apache/spark/sql/api/java/UDF6.java | 0 .../java/org/apache/spark/sql/api/java/UDF7.java | 0 .../java/org/apache/spark/sql/api/java/UDF8.java | 0 .../java/org/apache/spark/sql/api/java/UDF9.java | 0 .../apache/spark/sql/api/java/package-info.java| 0 .../spark/sql/catalyst/encoders/RowEncoder.scala | 2 +- 32 files changed, 909 insertions(+), 2 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 426709b8f18..2e8211a0966 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -20,7 +20,10 @@ package org.apache.spark.sql import scala.reflect.runtime.universe.{typeTag, TypeTag} import org.apache.spark.internal.Logging +import org.apache.spark.sql.api.java._ +import org.apache.spark.sql.connect.common.UdfUtils import org.apache.spark.sql.expressions.{ScalarUserDefinedFunction, UserDefinedFunction} +import org.apache.spark.sql.types.DataType /** * Functions for registering user-defined functions. Use `SparkSession.udf` to access this: @@ -1024,5 +1027,268 @@ class UDFRegistration(session: SparkSession) extends Logging { typeTag[A22]) register(name, udf) } + + // (0 to 22).foreach { i => + //val extTypeArgs = (0 to i).map(_ => "_").mkString(", ") +
[spark] branch branch-3.5 updated: [SPARK-43744][CONNECT] Fix class loading problem caused by stub user classes not found on the server classpath
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 408c3df01d2 [SPARK-43744][CONNECT] Fix class loading problem caused by stub user classes not found on the server classpath 408c3df01d2 is described below commit 408c3df01d20bb539929bdfdb236f899c428a12e Author: Zhen Li AuthorDate: Fri Jul 28 22:59:07 2023 -0400 [SPARK-43744][CONNECT] Fix class loading problem caused by stub user classes not found on the server classpath ### What changes were proposed in this pull request? This PR introduces a stub class loader for unpacking Scala UDFs in the driver and the executor. When encountering user classes that are not found on the server session classpath, the stub class loader would try to stub the class. This solves the problem that when serializing UDFs, Java serializer might include unnecessary user code e.g. User classes used in the lambda definition signatures in the same class where the UDF is defined. If the user code is actually needed to execute the UDF, we will return an error message to suggest the user to add the missing classes using the `addArtifact` method. ### Why are the changes needed? To enhance the user experience of UDF. This PR should be merged to master and 3.5. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added test both for Scala 2.12 & 2.13 4 tests in SparkSessionE2ESuite still fail to run with maven after the fix because the client test jar is installed on the system classpath (added using --jar at server start), the stub classloader can only stub classes missing from the session classpath (added using `session.addArtifact`). Moving the test jar to the session classpath causes failures in tests for `flatMapGroupsWithState` (SPARK-44576). Finish moving the test jar to session classpath once `flatMapGroupsWithState` test failures are fixed. Closes #42069 from zhenlineo/ref-spark-result. Authored-by: Zhen Li Signed-off-by: Herman van Hovell (cherry picked from commit 6d0fed9a18ff87e73fdf1ee46b6b0d2df8dd5a1b) Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 2 +- .../sql/expressions/UserDefinedFunction.scala | 2 +- .../jvm/src/test/resources/StubClassDummyUdf.scala | 56 + .../connect/client/jvm/src/test/resources/udf2.12 | Bin 0 -> 1520 bytes .../client/jvm/src/test/resources/udf2.12.jar | Bin 0 -> 5332 bytes .../connect/client/jvm/src/test/resources/udf2.13 | Bin 0 -> 1630 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 0 -> 5674 bytes .../connect/client/UDFClassLoadingE2ESuite.scala | 83 + .../connect/client/util/IntegrationTestUtils.scala | 2 +- .../connect/client/util/RemoteSparkSession.scala | 2 +- .../artifact/SparkConnectArtifactManager.scala | 17 ++- .../sql/connect/planner/SparkConnectPlanner.scala | 23 +++- connector/connect/server/src/test/resources/udf| Bin 0 -> 973 bytes .../connect/server/src/test/resources/udf_noA.jar | Bin 0 -> 5545 bytes .../connect/artifact/StubClassLoaderSuite.scala| 132 + .../spark/util/ChildFirstURLClassLoader.java | 9 ++ .../scala/org/apache/spark/executor/Executor.scala | 86 +++--- .../org/apache/spark/internal/config/package.scala | 14 +++ .../org/apache/spark/util/StubClassLoader.scala| 79 19 files changed, 480 insertions(+), 27 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index d1832e65f3e..4b3de91b56f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -554,7 +554,7 @@ class SparkSession private[sql] ( val command = proto.Command.newBuilder().setRegisterFunction(udf).build() val plan = proto.Plan.newBuilder().setCommand(command).build() -client.execute(plan) +client.execute(plan).asScala.foreach(_ => ()) } @DeveloperApi diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 18aef8a2e4c..e5c89d90c19 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -92,7 +92,7 @@ sealed abstract
[spark] branch master updated: [SPARK-43744][CONNECT] Fix class loading problem caused by stub user classes not found on the server classpath
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6d0fed9a18f [SPARK-43744][CONNECT] Fix class loading problem caused by stub user classes not found on the server classpath 6d0fed9a18f is described below commit 6d0fed9a18ff87e73fdf1ee46b6b0d2df8dd5a1b Author: Zhen Li AuthorDate: Fri Jul 28 22:59:07 2023 -0400 [SPARK-43744][CONNECT] Fix class loading problem caused by stub user classes not found on the server classpath ### What changes were proposed in this pull request? This PR introduces a stub class loader for unpacking Scala UDFs in the driver and the executor. When encountering user classes that are not found on the server session classpath, the stub class loader would try to stub the class. This solves the problem that when serializing UDFs, Java serializer might include unnecessary user code e.g. User classes used in the lambda definition signatures in the same class where the UDF is defined. If the user code is actually needed to execute the UDF, we will return an error message to suggest the user to add the missing classes using the `addArtifact` method. ### Why are the changes needed? To enhance the user experience of UDF. This PR should be merged to master and 3.5. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added test both for Scala 2.12 & 2.13 4 tests in SparkSessionE2ESuite still fail to run with maven after the fix because the client test jar is installed on the system classpath (added using --jar at server start), the stub classloader can only stub classes missing from the session classpath (added using `session.addArtifact`). Moving the test jar to the session classpath causes failures in tests for `flatMapGroupsWithState` (SPARK-44576). Finish moving the test jar to session classpath once `flatMapGroupsWithState` test failures are fixed. Closes #42069 from zhenlineo/ref-spark-result. Authored-by: Zhen Li Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 2 +- .../sql/expressions/UserDefinedFunction.scala | 2 +- .../jvm/src/test/resources/StubClassDummyUdf.scala | 56 + .../connect/client/jvm/src/test/resources/udf2.12 | Bin 0 -> 1520 bytes .../client/jvm/src/test/resources/udf2.12.jar | Bin 0 -> 5332 bytes .../connect/client/jvm/src/test/resources/udf2.13 | Bin 0 -> 1630 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 0 -> 5674 bytes .../connect/client/UDFClassLoadingE2ESuite.scala | 83 + .../connect/client/util/IntegrationTestUtils.scala | 2 +- .../connect/client/util/RemoteSparkSession.scala | 2 +- .../artifact/SparkConnectArtifactManager.scala | 17 ++- .../sql/connect/planner/SparkConnectPlanner.scala | 23 +++- connector/connect/server/src/test/resources/udf| Bin 0 -> 973 bytes .../connect/server/src/test/resources/udf_noA.jar | Bin 0 -> 5545 bytes .../connect/artifact/StubClassLoaderSuite.scala| 132 + .../spark/util/ChildFirstURLClassLoader.java | 9 ++ .../scala/org/apache/spark/executor/Executor.scala | 86 +++--- .../org/apache/spark/internal/config/package.scala | 14 +++ .../org/apache/spark/util/StubClassLoader.scala| 79 19 files changed, 480 insertions(+), 27 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index d1832e65f3e..4b3de91b56f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -554,7 +554,7 @@ class SparkSession private[sql] ( val command = proto.Command.newBuilder().setRegisterFunction(udf).build() val plan = proto.Plan.newBuilder().setCommand(command).build() -client.execute(plan) +client.execute(plan).asScala.foreach(_ => ()) } @DeveloperApi diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 18aef8a2e4c..e5c89d90c19 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -92,7 +92,7 @@ sealed abstract class UserDefinedFunction { /** * Holder class for a scalar user-defined function and it's input/output
[spark] branch branch-3.5 updated: [SPARK-44574][SQL][CONNECT] Errors that moved into sq/api should also use AnalysisException
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new d01821e0a99 [SPARK-44574][SQL][CONNECT] Errors that moved into sq/api should also use AnalysisException d01821e0a99 is described below commit d01821e0a99f527b1c151c19b0f44c26121ad57d Author: Rui Wang AuthorDate: Fri Jul 28 22:47:00 2023 -0400 [SPARK-44574][SQL][CONNECT] Errors that moved into sq/api should also use AnalysisException ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/42130 moved AnalysisException to sql/api. So we can use AnalysisException for those errors that were moved into sql/api but changed to SparkException. ### Why are the changes needed? Restore to previous behavior. ### Does this PR introduce _any_ user-facing change? No. This PR recovers the behaviors to the past which users should see AnalysisException upon many cases. ### How was this patch tested? Existing UT. Closes #42190 from amaliujia/convert_back_errors. Authored-by: Rui Wang Signed-off-by: Herman van Hovell (cherry picked from commit 3761b7d65bd1e21b5a7c5966c2d03a0fe4a0b982) Signed-off-by: Herman van Hovell --- .../apache/spark/sql/errors/DataTypeErrors.scala | 53 ++ .../spark/sql/errors/QueryCompilationErrors.scala | 7 +-- .../apache/spark/sql/types/StructTypeSuite.scala | 26 +-- .../apache/spark/sql/CharVarcharTestSuite.scala| 28 +++- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../org/apache/spark/sql/JsonFunctionsSuite.scala | 6 +-- .../spark/sql/connector/AlterTableTests.scala | 2 +- .../sql/errors/QueryCompilationErrorsSuite.scala | 4 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../sql/test/DataFrameReaderWriterSuite.scala | 12 ++--- 10 files changed, 68 insertions(+), 76 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala index fcc3086b573..7a34a386cd8 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala @@ -17,8 +17,10 @@ package org.apache.spark.sql.errors import org.apache.spark.{SparkArithmeticException, SparkException, SparkIllegalArgumentException, SparkNumberFormatException, SparkRuntimeException, SparkUnsupportedOperationException} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.trees.{Origin, SQLQueryContext} import org.apache.spark.sql.catalyst.util.QuotingUtils +import org.apache.spark.sql.catalyst.util.QuotingUtils.toSQLSchema import org.apache.spark.sql.types.{DataType, Decimal, StringType} import org.apache.spark.unsafe.types.UTF8String @@ -97,49 +99,45 @@ private[sql] object DataTypeErrors extends DataTypeErrorsBase { } def schemaFailToParseError(schema: String, e: Throwable): Throwable = { -new SparkException( +new AnalysisException( errorClass = "INVALID_SCHEMA.PARSE_ERROR", messageParameters = Map( -"inputSchema" -> QuotingUtils.toSQLSchema(schema), +"inputSchema" -> toSQLSchema(schema), "reason" -> e.getMessage ), - cause = e) + cause = Some(e)) } def invalidDayTimeIntervalType(startFieldName: String, endFieldName: String): Throwable = { -new SparkException( +new AnalysisException( errorClass = "_LEGACY_ERROR_TEMP_1224", messageParameters = Map( "startFieldName" -> startFieldName, -"endFieldName" -> endFieldName), - cause = null) +"endFieldName" -> endFieldName)) } def invalidDayTimeField(field: Byte, supportedIds: Seq[String]): Throwable = { -new SparkException( +new AnalysisException( errorClass = "_LEGACY_ERROR_TEMP_1223", messageParameters = Map( "field" -> field.toString, -"supportedIds" -> supportedIds.mkString(", ")), - cause = null) +"supportedIds" -> supportedIds.mkString(", "))) } def invalidYearMonthField(field: Byte, supportedIds: Seq[String]): Throwable = { -new SparkException( +new AnalysisException( errorClass = "_LEGACY_ERROR_TEMP_1225", messageParameters = Map( "field" -> field.toString, -"supportedIds" -> supportedIds.mkString(", ")), - cause = null) +"supportedIds" -> supportedIds.mkString(", &
[spark] branch master updated (85a4d1e56e8 -> 3761b7d65bd)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 85a4d1e56e8 [SPARK-41400][CONNECT] Remove Connect Client Catalyst Dependency add 3761b7d65bd [SPARK-44574][SQL][CONNECT] Errors that moved into sq/api should also use AnalysisException No new revisions were added by this update. Summary of changes: .../apache/spark/sql/errors/DataTypeErrors.scala | 53 ++ .../spark/sql/errors/QueryCompilationErrors.scala | 7 +-- .../apache/spark/sql/types/StructTypeSuite.scala | 26 +-- .../apache/spark/sql/CharVarcharTestSuite.scala| 28 +++- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../org/apache/spark/sql/JsonFunctionsSuite.scala | 6 +-- .../spark/sql/connector/AlterTableTests.scala | 2 +- .../sql/errors/QueryCompilationErrorsSuite.scala | 4 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../sql/test/DataFrameReaderWriterSuite.scala | 12 ++--- 10 files changed, 68 insertions(+), 76 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (72af2c0fbc6 -> 85a4d1e56e8)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 72af2c0fbc6 [SPARK-44585][MLLIB] Fix warning condition in MLLib RankingMetrics ndcgAk add 85a4d1e56e8 [SPARK-41400][CONNECT] Remove Connect Client Catalyst Dependency No new revisions were added by this update. Summary of changes: common/network-common/pom.xml | 3 +- .../apache/spark/network/sasl/SparkSaslSuite.java | 5 +- .../org/apache/spark/network/util/ByteUnit.java| 0 .../org/apache/spark/network/util/JavaUtils.java | 66 ++--- .../org/apache/spark/util/SparkErrorUtils.scala| 53 +++- .../org/apache/spark/util/SparkFileUtils.scala | 80 +- .../org/apache/spark/util/SparkSerDeUtils.scala| 9 +- connector/connect/client/jvm/pom.xml | 14 +- .../scala/org/apache/spark/sql/SparkSession.scala | 6 +- .../connect/client/arrow/ArrowDeserializer.scala | 6 +- .../sql/connect/client/arrow/ArrowSerializer.scala | 19 +- .../connect/client/arrow/ArrowVectorReader.scala | 15 +- .../org/apache/spark/sql/protobuf/functions.scala | 15 +- .../org/apache/spark/sql/ClientE2ETestSuite.scala | 7 +- .../spark/sql/DataFrameNaFunctionSuite.scala | 6 +- .../apache/spark/sql/PlanGenerationTestSuite.scala | 6 +- .../scala/org/apache/spark/sql/SQLHelper.scala | 11 +- .../apache/spark/sql/SQLImplicitsTestSuite.scala | 31 ++- .../apache/spark/sql/SparkSessionE2ESuite.scala| 16 +- .../sql/UserDefinedFunctionE2ETestSuite.scala | 15 +- .../spark/sql/UserDefinedFunctionSuite.scala | 4 +- .../sql/connect/client/ClassFinderSuite.scala | 6 +- .../connect/client/arrow/ArrowEncoderSuite.scala | 6 +- .../connect/client/util/IntegrationTestUtils.scala | 11 +- .../spark/sql/connect/client/util/QueryTest.scala | 4 +- .../spark/sql/streaming/StreamingQuerySuite.scala | 6 +- connector/connect/common/pom.xml | 8 +- .../common/LiteralValueProtoConverter.scala| 14 +- .../main/scala/org/apache/spark/util/Utils.scala | 132 +- .../catalyst/plans/logical/LogicalGroupState.scala | 7 +- .../sql/catalyst/util/SparkIntervalUtils.scala | 287 - .../spark/sql/catalyst/util/StringUtils.scala | 20 ++ .../spark/sql/errors/CompilationErrors.scala | 54 .../org/apache/spark/sql/internal/SqlApiConf.scala | 3 + .../org/apache/spark/sql/types/UpCastRule.scala| 86 ++ .../sql/catalyst/analysis/AnsiTypeCoercion.scala | 2 +- .../spark/sql/catalyst/analysis/TypeCoercion.scala | 12 +- .../spark/sql/catalyst/expressions/Cast.scala | 47 +--- .../sql/catalyst/expressions/ToStringBase.scala| 6 +- .../spark/sql/catalyst/plans/logical/object.scala | 3 - .../spark/sql/catalyst/util/IntervalUtils.scala| 260 --- .../apache/spark/sql/catalyst/util/package.scala | 21 +- .../spark/sql/errors/QueryCompilationErrors.scala | 33 +-- .../org/apache/spark/sql/internal/SQLConf.scala| 6 +- .../sql/catalyst/expressions/CastSuiteBase.scala | 2 +- 45 files changed, 751 insertions(+), 672 deletions(-) rename common/{network-common => utils}/src/main/java/org/apache/spark/network/util/ByteUnit.java (100%) rename common/{network-common => utils}/src/main/java/org/apache/spark/network/util/JavaUtils.java (89%) copy connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java => sql/api/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalGroupState.scala (86%) create mode 100644 sql/api/src/main/scala/org/apache/spark/sql/errors/CompilationErrors.scala create mode 100644 sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.5 updated: [SPARK-41400][CONNECT] Remove Connect Client Catalyst Dependency
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 5412fb0590e [SPARK-41400][CONNECT] Remove Connect Client Catalyst Dependency 5412fb0590e is described below commit 5412fb0590e55d635e9e31887ec5c72d10011899 Author: Herman van Hovell AuthorDate: Fri Jul 28 21:30:51 2023 -0400 [SPARK-41400][CONNECT] Remove Connect Client Catalyst Dependency ### What changes were proposed in this pull request? This PR decouples the Spark Connect Scala Client from Catalyst, it now used SQL API module instead. There were quite a few changes we still needed to make: - For testing we needed a bunch of utilities. I have moved these to common-utils. - I have moved bits and pieces of IntervalUtils to SparkIntervalUtils. - A lot of small fixes. ### Why are the changes needed? This reduces the client's dependency tree from ~300 MB of deps to ~30MB. This makes it easier to use the client when you are developing connect applications. On top of this the reduced dependency graph also means folks will be less affected by the clients' classpath. ### Does this PR introduce _any_ user-facing change? Yes. It changes the classpath exposed by the Spark Connect Scala Client. ### How was this patch tested? Existing tests. Closes #42184 from hvanhovell/SPARK-41400-v1. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 85a4d1e56e80d85dad9b8945c67287927eb379f6) Signed-off-by: Herman van Hovell --- common/network-common/pom.xml | 3 +- .../apache/spark/network/sasl/SparkSaslSuite.java | 5 +- .../org/apache/spark/network/util/ByteUnit.java| 0 .../org/apache/spark/network/util/JavaUtils.java | 66 ++--- .../org/apache/spark/util/SparkErrorUtils.scala| 53 +++- .../org/apache/spark/util/SparkFileUtils.scala | 80 +- .../org/apache/spark/util/SparkSerDeUtils.scala| 9 +- connector/connect/client/jvm/pom.xml | 14 +- .../scala/org/apache/spark/sql/SparkSession.scala | 6 +- .../connect/client/arrow/ArrowDeserializer.scala | 6 +- .../sql/connect/client/arrow/ArrowSerializer.scala | 19 +- .../connect/client/arrow/ArrowVectorReader.scala | 15 +- .../org/apache/spark/sql/protobuf/functions.scala | 15 +- .../org/apache/spark/sql/ClientE2ETestSuite.scala | 7 +- .../spark/sql/DataFrameNaFunctionSuite.scala | 6 +- .../apache/spark/sql/PlanGenerationTestSuite.scala | 6 +- .../scala/org/apache/spark/sql/SQLHelper.scala | 11 +- .../apache/spark/sql/SQLImplicitsTestSuite.scala | 31 ++- .../apache/spark/sql/SparkSessionE2ESuite.scala| 16 +- .../sql/UserDefinedFunctionE2ETestSuite.scala | 15 +- .../spark/sql/UserDefinedFunctionSuite.scala | 4 +- .../sql/connect/client/ClassFinderSuite.scala | 6 +- .../connect/client/arrow/ArrowEncoderSuite.scala | 6 +- .../connect/client/util/IntegrationTestUtils.scala | 11 +- .../spark/sql/connect/client/util/QueryTest.scala | 4 +- .../spark/sql/streaming/StreamingQuerySuite.scala | 6 +- connector/connect/common/pom.xml | 8 +- .../common/LiteralValueProtoConverter.scala| 14 +- .../main/scala/org/apache/spark/util/Utils.scala | 132 +- .../catalyst/plans/logical/LogicalGroupState.scala | 16 +- .../sql/catalyst/util/SparkIntervalUtils.scala | 287 - .../spark/sql/catalyst/util/StringUtils.scala | 20 ++ .../spark/sql/errors/CompilationErrors.scala | 54 .../org/apache/spark/sql/internal/SqlApiConf.scala | 3 + .../org/apache/spark/sql/types/UpCastRule.scala| 86 ++ .../sql/catalyst/analysis/AnsiTypeCoercion.scala | 2 +- .../spark/sql/catalyst/analysis/TypeCoercion.scala | 12 +- .../spark/sql/catalyst/expressions/Cast.scala | 47 +--- .../sql/catalyst/expressions/ToStringBase.scala| 6 +- .../spark/sql/catalyst/plans/logical/object.scala | 3 - .../spark/sql/catalyst/util/IntervalUtils.scala| 260 --- .../apache/spark/sql/catalyst/util/package.scala | 21 +- .../spark/sql/errors/QueryCompilationErrors.scala | 33 +-- .../org/apache/spark/sql/internal/SQLConf.scala| 6 +- .../sql/catalyst/expressions/CastSuiteBase.scala | 2 +- 45 files changed, 751 insertions(+), 681 deletions(-) diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 8a63e999c53..2b43f9ce98a 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -150,7 +150,8 @@ org.apache.spark - spark-tags_${scala.binary.version} + spark-common-utils_${scala.binary.version} + ${proje
[spark] branch 3.5 created (now 3dcee0ccf7e)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch 3.5 in repository https://gitbox.apache.org/repos/asf/spark.git at 3dcee0ccf7e [SPARK-44538][CONNECT][SQL] Reinstate Row.jsonValue and friends No new revisions were added by this update. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.5 updated: [SPARK-44531][CONNECT][SQL] Move encoder inference to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new e0c8f14ce53 [SPARK-44531][CONNECT][SQL] Move encoder inference to sql/api e0c8f14ce53 is described below commit e0c8f14ce53080e2863c076b7912239bee35003e Author: Herman van Hovell AuthorDate: Wed Jul 26 07:15:27 2023 -0400 [SPARK-44531][CONNECT][SQL] Move encoder inference to sql/api ### What changes were proposed in this pull request? This PR move encoder inference (ScalaReflection/RowEncoder/JavaTypeInference) into sql/api. ### Why are the changes needed? We want to use encoder inference in the spark connect scala client. The client's dependency to catalyst is going away, so we need to move this. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #42134 from hvanhovell/SPARK-44531. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 071feabbd4325504332679dfa620bc5ee4359370) Signed-off-by: Herman van Hovell --- .../spark/ml/source/image/ImageFileFormat.scala| 4 +- .../spark/ml/source/libsvm/LibSVMRelation.scala| 4 +- project/MimaExcludes.scala | 3 + sql/api/pom.xml| 4 ++ .../java/org/apache/spark/sql/types/DataTypes.java | 0 .../apache/spark/sql/types/SQLUserDefinedType.java | 0 .../scala/org/apache/spark/sql/SqlApiConf.scala| 2 + .../spark/sql/catalyst/JavaTypeInference.scala | 9 ++- .../spark/sql/catalyst/ScalaReflection.scala | 13 ++-- .../apache/spark/sql/catalyst/WalkedTypePath.scala | 0 .../spark/sql/catalyst/encoders/RowEncoder.scala | 19 ++ .../spark/sql/errors/DataTypeErrorsBase.scala | 8 ++- .../apache/spark/sql/errors/EncoderErrors.scala| 74 ++ sql/catalyst/pom.xml | 5 -- .../sql/catalyst/encoders/ExpressionEncoder.scala | 8 ++- .../spark/sql/catalyst/plans/logical/object.scala | 8 +-- .../spark/sql/errors/QueryExecutionErrors.scala| 60 +- .../org/apache/spark/sql/internal/SQLConf.scala| 2 +- .../spark/sql/CalendarIntervalBenchmark.scala | 4 +- .../scala/org/apache/spark/sql/HashBenchmark.scala | 4 +- .../spark/sql/UnsafeProjectionBenchmark.scala | 4 +- .../sql/catalyst/encoders/RowEncoderSuite.scala| 50 +++ .../expressions/HashExpressionsSuite.scala | 4 +- .../expressions/ObjectExpressionsSuite.scala | 2 +- .../optimizer/ObjectSerializerPruningSuite.scala | 4 +- .../catalyst/util/ArrayDataIndexedSeqSuite.scala | 4 +- .../spark/sql/catalyst/util/UnsafeArraySuite.scala | 6 +- .../main/scala/org/apache/spark/sql/Dataset.scala | 6 +- .../scala/org/apache/spark/sql/SparkSession.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../execution/datasources/DataSourceStrategy.scala | 4 +- .../sql/execution/datasources/jdbc/JdbcUtils.scala | 4 +- .../execution/datasources/v2/V2CommandExec.scala | 4 +- .../FlatMapGroupsInPandasWithStateExec.scala | 4 +- .../execution/streaming/MicroBatchExecution.scala | 4 +- .../sql/execution/streaming/sources/memory.scala | 4 +- .../spark/sql/DataFrameSessionWindowingSuite.scala | 4 +- .../org/apache/spark/sql/DataFrameSuite.scala | 8 +-- .../spark/sql/DataFrameTimeWindowingSuite.scala| 4 +- .../spark/sql/DatasetOptimizationSuite.scala | 4 +- .../scala/org/apache/spark/sql/DatasetSuite.scala | 8 +-- .../spark/sql/execution/GroupedIteratorSuite.scala | 8 +-- .../binaryfile/BinaryFileFormatSuite.scala | 4 +- .../streaming/sources/ForeachBatchSinkSuite.scala | 4 +- .../apache/spark/sql/streaming/StreamTest.scala| 4 +- 45 files changed, 205 insertions(+), 184 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageFileFormat.scala b/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageFileFormat.scala index 206ce6f0675..bf6e6b8eec0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageFileFormat.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageFileFormat.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.ml.image.ImageSchema import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.sources.{DataSourceReg
[spark] branch master updated: [SPARK-44531][CONNECT][SQL] Move encoder inference to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 071feabbd43 [SPARK-44531][CONNECT][SQL] Move encoder inference to sql/api 071feabbd43 is described below commit 071feabbd4325504332679dfa620bc5ee4359370 Author: Herman van Hovell AuthorDate: Wed Jul 26 07:15:27 2023 -0400 [SPARK-44531][CONNECT][SQL] Move encoder inference to sql/api ### What changes were proposed in this pull request? This PR move encoder inference (ScalaReflection/RowEncoder/JavaTypeInference) into sql/api. ### Why are the changes needed? We want to use encoder inference in the spark connect scala client. The client's dependency to catalyst is going away, so we need to move this. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #42134 from hvanhovell/SPARK-44531. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../spark/ml/source/image/ImageFileFormat.scala| 4 +- .../spark/ml/source/libsvm/LibSVMRelation.scala| 4 +- project/MimaExcludes.scala | 3 + sql/api/pom.xml| 4 ++ .../java/org/apache/spark/sql/types/DataTypes.java | 0 .../apache/spark/sql/types/SQLUserDefinedType.java | 0 .../scala/org/apache/spark/sql/SqlApiConf.scala| 2 + .../spark/sql/catalyst/JavaTypeInference.scala | 9 ++- .../spark/sql/catalyst/ScalaReflection.scala | 13 ++-- .../apache/spark/sql/catalyst/WalkedTypePath.scala | 0 .../spark/sql/catalyst/encoders/RowEncoder.scala | 19 ++ .../spark/sql/errors/DataTypeErrorsBase.scala | 8 ++- .../apache/spark/sql/errors/EncoderErrors.scala| 74 ++ sql/catalyst/pom.xml | 5 -- .../sql/catalyst/encoders/ExpressionEncoder.scala | 8 ++- .../spark/sql/catalyst/plans/logical/object.scala | 8 +-- .../spark/sql/errors/QueryExecutionErrors.scala| 60 +- .../org/apache/spark/sql/internal/SQLConf.scala| 2 +- .../spark/sql/CalendarIntervalBenchmark.scala | 4 +- .../scala/org/apache/spark/sql/HashBenchmark.scala | 4 +- .../spark/sql/UnsafeProjectionBenchmark.scala | 4 +- .../sql/catalyst/encoders/RowEncoderSuite.scala| 50 +++ .../expressions/HashExpressionsSuite.scala | 4 +- .../expressions/ObjectExpressionsSuite.scala | 2 +- .../optimizer/ObjectSerializerPruningSuite.scala | 4 +- .../catalyst/util/ArrayDataIndexedSeqSuite.scala | 4 +- .../spark/sql/catalyst/util/UnsafeArraySuite.scala | 6 +- .../main/scala/org/apache/spark/sql/Dataset.scala | 6 +- .../scala/org/apache/spark/sql/SparkSession.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../execution/datasources/DataSourceStrategy.scala | 4 +- .../sql/execution/datasources/jdbc/JdbcUtils.scala | 4 +- .../execution/datasources/v2/V2CommandExec.scala | 4 +- .../FlatMapGroupsInPandasWithStateExec.scala | 4 +- .../execution/streaming/MicroBatchExecution.scala | 4 +- .../sql/execution/streaming/sources/memory.scala | 4 +- .../spark/sql/DataFrameSessionWindowingSuite.scala | 4 +- .../org/apache/spark/sql/DataFrameSuite.scala | 8 +-- .../spark/sql/DataFrameTimeWindowingSuite.scala| 4 +- .../spark/sql/DatasetOptimizationSuite.scala | 4 +- .../scala/org/apache/spark/sql/DatasetSuite.scala | 8 +-- .../spark/sql/execution/GroupedIteratorSuite.scala | 8 +-- .../binaryfile/BinaryFileFormatSuite.scala | 4 +- .../streaming/sources/ForeachBatchSinkSuite.scala | 4 +- .../apache/spark/sql/streaming/StreamTest.scala| 4 +- 45 files changed, 205 insertions(+), 184 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageFileFormat.scala b/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageFileFormat.scala index 206ce6f0675..bf6e6b8eec0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageFileFormat.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageFileFormat.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.ml.image.ImageSchema import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.sources.{DataSourceRegister, Filter} @@ -90,7 +90,7 @@ private[image] class ImageFileFormat extends FileFormat with DataSourceRegister
[spark] branch branch-3.5 updated: [SPARK-44535][CONNECT][SQL] Move required Streaming API to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 95a59223ce7 [SPARK-44535][CONNECT][SQL] Move required Streaming API to sql/api 95a59223ce7 is described below commit 95a59223ce79174c9a605a4305082a1211ccb7e8 Author: Herman van Hovell AuthorDate: Tue Jul 25 23:57:14 2023 -0400 [SPARK-44535][CONNECT][SQL] Move required Streaming API to sql/api ### What changes were proposed in this pull request? This PR moves a bunch streaming classed to the SQL/API project. ### Why are the changes needed? This is needed to disconnect the Spark Connect Scala Client from catalyst. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing Tests. Closes #42140 from hvanhovell/SPARK-44535. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 03d0ecc68365565e921ac6bc0f253ee67d2042d7) Signed-off-by: Herman van Hovell --- .../spark/sql/streaming/DataStreamWriter.scala | 3 +-- .../sql/streaming/StreamingQueryListener.scala | 3 +-- .../sql/streaming/StreamingQueryManager.scala | 6 ++--- .../CheckConnectJvmClientCompatibility.scala | 18 +++ dev/checkstyle-suppressions.xml| 4 ++-- project/MimaExcludes.scala | 5 +++- .../spark/sql/streaming/GroupStateTimeout.java | 0 .../org/apache/spark/sql/streaming/OutputMode.java | 0 .../expressions/{GenericRow.scala => rows.scala} | 10 .../logical/groupStateTimeouts.scala} | 27 +- .../catalyst/streaming/InternalOutputModes.scala | 0 .../spark/sql/catalyst/expressions/rows.scala | 9 .../spark/sql/catalyst/plans/logical/object.scala | 5 13 files changed, 45 insertions(+), 45 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index ad76ab4a1bc..b395a2d073d 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -38,7 +38,6 @@ import org.apache.spark.sql.execution.streaming.OneTimeTrigger import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger import org.apache.spark.sql.types.NullType import org.apache.spark.util.SparkSerDeUtils -import org.apache.spark.util.Utils /** * Interface used to write a streaming `Dataset` to external storage systems (e.g. file systems, @@ -240,7 +239,7 @@ final class DataStreamWriter[T] private[sql] (ds: Dataset[T]) extends Logging { */ @Evolving def foreachBatch(function: (Dataset[T], Long) => Unit): DataStreamWriter[T] = { -val serializedFn = Utils.serialize(function) +val serializedFn = SparkSerDeUtils.serialize(function) sinkBuilder.getForeachBatchBuilder.getScalaFunctionBuilder .setPayload(ByteString.copyFrom(serializedFn)) .setOutputType(DataTypeProtoConverter.toConnectProtoType(NullType)) // Unused. diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index 8cef421becd..e2f3be02ad3 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -25,7 +25,6 @@ import org.json4s.JsonDSL.{jobject2assoc, pair2Assoc} import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.annotation.Evolving -import org.apache.spark.scheduler.SparkListenerEvent /** * Interface for listening to events related to [[StreamingQuery StreamingQueries]]. @@ -116,7 +115,7 @@ object StreamingQueryListener extends Serializable { * @since 3.5.0 */ @Evolving - trait Event extends SparkListenerEvent + trait Event /** * Event representing the start of a query diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 8f9e768d23f..91744460440 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -31,7 +31,7 @@
[spark] branch master updated: [SPARK-44535][CONNECT][SQL] Move required Streaming API to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 03d0ecc6836 [SPARK-44535][CONNECT][SQL] Move required Streaming API to sql/api 03d0ecc6836 is described below commit 03d0ecc68365565e921ac6bc0f253ee67d2042d7 Author: Herman van Hovell AuthorDate: Tue Jul 25 23:57:14 2023 -0400 [SPARK-44535][CONNECT][SQL] Move required Streaming API to sql/api ### What changes were proposed in this pull request? This PR moves a bunch streaming classed to the SQL/API project. ### Why are the changes needed? This is needed to disconnect the Spark Connect Scala Client from catalyst. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing Tests. Closes #42140 from hvanhovell/SPARK-44535. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../spark/sql/streaming/DataStreamWriter.scala | 3 +-- .../sql/streaming/StreamingQueryListener.scala | 3 +-- .../sql/streaming/StreamingQueryManager.scala | 6 ++--- .../CheckConnectJvmClientCompatibility.scala | 18 +++ dev/checkstyle-suppressions.xml| 4 ++-- project/MimaExcludes.scala | 5 +++- .../spark/sql/streaming/GroupStateTimeout.java | 0 .../org/apache/spark/sql/streaming/OutputMode.java | 0 .../expressions/{GenericRow.scala => rows.scala} | 10 .../logical/groupStateTimeouts.scala} | 27 +- .../catalyst/streaming/InternalOutputModes.scala | 0 .../spark/sql/catalyst/expressions/rows.scala | 9 .../spark/sql/catalyst/plans/logical/object.scala | 5 13 files changed, 45 insertions(+), 45 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index ad76ab4a1bc..b395a2d073d 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -38,7 +38,6 @@ import org.apache.spark.sql.execution.streaming.OneTimeTrigger import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger import org.apache.spark.sql.types.NullType import org.apache.spark.util.SparkSerDeUtils -import org.apache.spark.util.Utils /** * Interface used to write a streaming `Dataset` to external storage systems (e.g. file systems, @@ -240,7 +239,7 @@ final class DataStreamWriter[T] private[sql] (ds: Dataset[T]) extends Logging { */ @Evolving def foreachBatch(function: (Dataset[T], Long) => Unit): DataStreamWriter[T] = { -val serializedFn = Utils.serialize(function) +val serializedFn = SparkSerDeUtils.serialize(function) sinkBuilder.getForeachBatchBuilder.getScalaFunctionBuilder .setPayload(ByteString.copyFrom(serializedFn)) .setOutputType(DataTypeProtoConverter.toConnectProtoType(NullType)) // Unused. diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index 8cef421becd..e2f3be02ad3 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -25,7 +25,6 @@ import org.json4s.JsonDSL.{jobject2assoc, pair2Assoc} import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.annotation.Evolving -import org.apache.spark.scheduler.SparkListenerEvent /** * Interface for listening to events related to [[StreamingQuery StreamingQueries]]. @@ -116,7 +115,7 @@ object StreamingQueryListener extends Serializable { * @since 3.5.0 */ @Evolving - trait Event extends SparkListenerEvent + trait Event /** * Event representing the start of a query diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 8f9e768d23f..91744460440 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -31,7 +31,7 @@ import org.apache.spark.connect.proto.StreamingQueryManagerCommandResult import org.apache.spark.internal.Logging
[spark] branch branch-3.5 updated: [SPARK-44532][CONNECT][SQL] Move ArrowUtils to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 3ee3a48ba9c [SPARK-44532][CONNECT][SQL] Move ArrowUtils to sql/api 3ee3a48ba9c is described below commit 3ee3a48ba9c45b884dd40d0e7063be47e7164112 Author: Herman van Hovell AuthorDate: Tue Jul 25 23:51:12 2023 -0400 [SPARK-44532][CONNECT][SQL] Move ArrowUtils to sql/api Backport of https://github.com/apache/spark/commit/307e46cc4dfdad1f442e8c5c50ecb53c9ef7dc47 to branch-3.5. ## What changes were proposed in this pull request? This PR moves ArrowUtils to sql/api. One method used for configuring python's arrow runner has been moved to ArrowPythonRunner . ## Why are the changes needed? ArrowUtils is used by connect's direct Arrow encoding (and a lot of other things in sql). We want to remove the connect scala client's catalyst dependency. We need to move ArrowUtil in order to do so. ## Does this PR introduce any user-facing change? No ## How was this patch tested? Existing tests. Closes #42156 from hvanhovell/SPARK-44532-3.5. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../connect/client/arrow/ArrowDeserializer.scala | 4 +-- .../sql/connect/client/arrow/ArrowSerializer.scala | 4 +-- sql/api/pom.xml| 8 + .../org/apache/spark/sql/errors/ArrowErrors.scala | 39 ++ .../apache/spark/sql/errors/DataTypeErrors.scala | 6 .../org/apache/spark/sql/util/ArrowUtils.scala | 19 +++ sql/catalyst/pom.xml | 8 - .../spark/sql/errors/QueryExecutionErrors.scala| 20 --- .../spark/sql/execution/arrow/ArrowWriter.scala| 4 +-- .../org/apache/spark/sql/execution/Columnar.scala | 4 +-- .../execution/python/AggregateInPandasExec.scala | 3 +- .../sql/execution/python/ArrowEvalPythonExec.scala | 3 +- .../execution/python/ArrowEvalPythonUDTFExec.scala | 3 +- .../sql/execution/python/ArrowPythonRunner.scala | 12 +++ .../python/FlatMapCoGroupsInPandasExec.scala | 3 +- .../python/FlatMapGroupsInPandasExec.scala | 3 +- .../FlatMapGroupsInPandasWithStateExec.scala | 3 +- .../sql/execution/python/MapInBatchExec.scala | 3 +- .../sql/execution/python/WindowInPandasExec.scala | 3 +- 19 files changed, 85 insertions(+), 67 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala index 01aba9cb0ce..4177a88ba52 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.errors.{DataTypeErrors, QueryCompilationErrors} import org.apache.spark.sql.types.Decimal /** @@ -341,7 +341,7 @@ object ArrowDeserializers { } case (CalendarIntervalEncoder | _: UDTEncoder[_], _) => -throw QueryExecutionErrors.unsupportedDataTypeError(encoder.dataType) +throw DataTypeErrors.unsupportedDataTypeError(encoder.dataType) case _ => throw new RuntimeException( diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala index d29f90a6a19..9b39a75ceed 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.DefinedByConstructorParams import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils} -import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.errors.DataTypeErrors import org.apache.spark.sql.types.Decimal import org.apache.spark.sql.util.ArrowUtils @@ -439,7 +439,7 @@ object ArrowSerializer {
[spark] branch branch-3.5 updated: [SPARK-44530][CORE][CONNECT] Move SparkBuildInfo to common/util
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 766428f814a [SPARK-44530][CORE][CONNECT] Move SparkBuildInfo to common/util 766428f814a is described below commit 766428f814a65ee6be7033096445dd5656dbc9be Author: Herman van Hovell AuthorDate: Tue Jul 25 20:51:27 2023 -0400 [SPARK-44530][CORE][CONNECT] Move SparkBuildInfo to common/util ### What changes were proposed in this pull request? This PR moves `SparkBuildInfo` and the code that generates its properties to `common/util`. ### Why are the changes needed? We need `SparkBuildInfo` in the connect scala client and we are removing connect's dependency on `core`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. Closes #42133 from hvanhovell/SPARK-44530. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 35187995bf9eaaa57fd29b714a304355c3049c39) Signed-off-by: Herman van Hovell --- common/utils/pom.xml | 62 .../scala/org/apache/spark/SparkBuildInfo.scala| 65 + core/pom.xml | 62 core/src/main/scala/org/apache/spark/package.scala | 66 +++--- project/SparkBuild.scala | 39 +++-- 5 files changed, 158 insertions(+), 136 deletions(-) diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 36cfceed931..46ce3ced6ab 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -84,7 +84,69 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + +${project.basedir}/src/main/resources + + + +${project.build.directory}/extra-resources +true + + + +org.apache.maven.plugins +maven-antrun-plugin + + +choose-shell-and-script +validate + + run + + + true + + + + + + + + + + + +Shell to use for generating spark-version-info.properties file = + ${shell} + +Script to use for generating spark-version-info.properties file = + ${spark-build-info-script} + + + + + +generate-spark-build-info +generate-resources + + + + + + + + + + + + run + + + + diff --git a/common/utils/src/main/scala/org/apache/spark/SparkBuildInfo.scala b/common/utils/src/main/scala/org/apache/spark/SparkBuildInfo.scala new file mode 100644 index 000..23f671f9d76 --- /dev/null +++ b/common/utils/src/main/scala/org/apache/spark/SparkBuildInfo.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark + +import java.util.Properties + +object SparkBuildInfo { + + val ( +spark_version: String, +spark_branch: String, +spark_revision: String, +spark_build_user: String, +spark_repo_url: String, +spark_build_date: String, +spark_doc_root: String) = { + +val resourceStream = Thread.currentThread().getContextClassLoader. + getResourceAsStream("spark-version-info.properties") +if (resourceStream == null) { + throw new SparkException("Could not find spark-version-info.properties") +} + +try { + val unkno
[spark] branch master updated: [SPARK-44530][CORE][CONNECT] Move SparkBuildInfo to common/util
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 35187995bf9 [SPARK-44530][CORE][CONNECT] Move SparkBuildInfo to common/util 35187995bf9 is described below commit 35187995bf9eaaa57fd29b714a304355c3049c39 Author: Herman van Hovell AuthorDate: Tue Jul 25 20:51:27 2023 -0400 [SPARK-44530][CORE][CONNECT] Move SparkBuildInfo to common/util ### What changes were proposed in this pull request? This PR moves `SparkBuildInfo` and the code that generates its properties to `common/util`. ### Why are the changes needed? We need `SparkBuildInfo` in the connect scala client and we are removing connect's dependency on `core`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. Closes #42133 from hvanhovell/SPARK-44530. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- common/utils/pom.xml | 62 .../scala/org/apache/spark/SparkBuildInfo.scala| 65 + core/pom.xml | 62 core/src/main/scala/org/apache/spark/package.scala | 66 +++--- project/SparkBuild.scala | 39 +++-- 5 files changed, 158 insertions(+), 136 deletions(-) diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 20abad21243..2f2fee0cf41 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -84,7 +84,69 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + +${project.basedir}/src/main/resources + + + +${project.build.directory}/extra-resources +true + + + +org.apache.maven.plugins +maven-antrun-plugin + + +choose-shell-and-script +validate + + run + + + true + + + + + + + + + + + +Shell to use for generating spark-version-info.properties file = + ${shell} + +Script to use for generating spark-version-info.properties file = + ${spark-build-info-script} + + + + + +generate-spark-build-info +generate-resources + + + + + + + + + + + + run + + + + diff --git a/common/utils/src/main/scala/org/apache/spark/SparkBuildInfo.scala b/common/utils/src/main/scala/org/apache/spark/SparkBuildInfo.scala new file mode 100644 index 000..23f671f9d76 --- /dev/null +++ b/common/utils/src/main/scala/org/apache/spark/SparkBuildInfo.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark + +import java.util.Properties + +object SparkBuildInfo { + + val ( +spark_version: String, +spark_branch: String, +spark_revision: String, +spark_build_user: String, +spark_repo_url: String, +spark_build_date: String, +spark_doc_root: String) = { + +val resourceStream = Thread.currentThread().getContextClassLoader. + getResourceAsStream("spark-version-info.properties") +if (resourceStream == null) { + throw new SparkException("Could not find spark-version-info.properties") +} + +try { + val unknownProp = "" + val props = new Properties() + props.load(resourceStream) +
[spark] branch master updated: [SPARK-44532][CONNECT][SQL] Move ArrowUtils to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 307e46cc4df [SPARK-44532][CONNECT][SQL] Move ArrowUtils to sql/api 307e46cc4df is described below commit 307e46cc4dfdad1f442e8c5c50ecb53c9ef7dc47 Author: Herman van Hovell AuthorDate: Tue Jul 25 13:35:37 2023 -0400 [SPARK-44532][CONNECT][SQL] Move ArrowUtils to sql/api ### What changes were proposed in this pull request? This PR moves `ArrowUtils` to `sql/api`. One method used for configuring python's arrow runner has been moved to `ArrowPythonRunner `. ### Why are the changes needed? ArrowUtils is used by connect's direct Arrow encoding (and a lot of other things in sql). We want to remove the connect scala client's catalyst dependency. We need to move ArrowUtil in order to do so. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. Closes #42137 from hvanhovell/SPARK-44532. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../connect/client/arrow/ArrowDeserializer.scala | 4 +-- .../sql/connect/client/arrow/ArrowSerializer.scala | 4 +-- sql/api/pom.xml| 8 + .../org/apache/spark/sql/errors/ArrowErrors.scala | 39 ++ .../apache/spark/sql/errors/DataTypeErrors.scala | 6 .../org/apache/spark/sql/util/ArrowUtils.scala | 19 +++ sql/catalyst/pom.xml | 8 - .../spark/sql/errors/QueryExecutionErrors.scala| 20 --- .../spark/sql/execution/arrow/ArrowWriter.scala| 4 +-- .../org/apache/spark/sql/execution/Columnar.scala | 4 +-- .../execution/python/AggregateInPandasExec.scala | 3 +- .../sql/execution/python/ArrowEvalPythonExec.scala | 3 +- .../execution/python/ArrowEvalPythonUDTFExec.scala | 3 +- .../sql/execution/python/ArrowPythonRunner.scala | 12 +++ .../python/FlatMapCoGroupsInPandasExec.scala | 3 +- .../python/FlatMapGroupsInPandasExec.scala | 3 +- .../FlatMapGroupsInPandasWithStateExec.scala | 3 +- .../sql/execution/python/MapInBatchExec.scala | 3 +- .../python/WindowInPandasEvaluatorFactory.scala| 4 +-- 19 files changed, 86 insertions(+), 67 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala index 01aba9cb0ce..4177a88ba52 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.errors.{DataTypeErrors, QueryCompilationErrors} import org.apache.spark.sql.types.Decimal /** @@ -341,7 +341,7 @@ object ArrowDeserializers { } case (CalendarIntervalEncoder | _: UDTEncoder[_], _) => -throw QueryExecutionErrors.unsupportedDataTypeError(encoder.dataType) +throw DataTypeErrors.unsupportedDataTypeError(encoder.dataType) case _ => throw new RuntimeException( diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala index d29f90a6a19..9b39a75ceed 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.DefinedByConstructorParams import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils} -import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.errors.DataTypeErrors import org.apache.spark.sql.types.Decimal import org.apache.spark.sql.util.ArrowUtils @@ -439,7 +439,7 @@ object ArrowSerializer { } case (CalendarIntervalEncoder | _: UDTEncoder[_], _) => -throw QueryExecutionErrors.
[spark] branch branch-3.5 updated: [SPARK-44449][CONNECT] Upcasting for direct Arrow Deserialization
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 7783789b066 [SPARK-9][CONNECT] Upcasting for direct Arrow Deserialization 7783789b066 is described below commit 7783789b066d6d54845083a3d036f9b3f0a44486 Author: Herman van Hovell AuthorDate: Mon Jul 24 20:46:01 2023 -0400 [SPARK-9][CONNECT] Upcasting for direct Arrow Deserialization ### What changes were proposed in this pull request? This PR adds upcasting to direct Arrow deserialization for the Spark Connect Scala Client. This is implemented by decoupling leaf encoders from vector implementations, instead all leaf encoders are now tied to an `ArrowVectorReader` instance that will allow us to read the data we need from an arbitrary vector type, provided we can read this data without data loss (this is both checked at runtime and compile time). ### Why are the changes needed? Direct Arrow deserialization did not support upcasting yet. Not supporting this would be a regression for connect compared to SPARK 3.4. ### Does this PR introduce _any_ user-facing change? Yes, it brings arrow encoders up to par with the existing catalyst encoding framework. ### How was this patch tested? Added tests to `ArrowEncoderSuite`. Re-enabled tests that relied on upcasting. Closes #42076 from hvanhovell/SPARK-9. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit a6ac63d14b56d939dda1aa2a8e74308efc8e1b93) Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 5 +- .../spark/sql/connect/client/SparkResult.scala | 6 +- .../connect/client/arrow/ArrowDeserializer.scala | 221 ++ .../connect/client/arrow/ArrowVectorReader.scala | 259 + .../org/apache/spark/sql/ClientE2ETestSuite.scala | 10 +- .../sql/KeyValueGroupedDatasetE2ETestSuite.scala | 5 +- .../connect/client/arrow/ArrowEncoderSuite.scala | 217 +++-- 7 files changed, 595 insertions(+), 128 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index b37e3884038..161b5a0217e 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -126,7 +126,6 @@ class SparkSession private[sql] ( private def createDataset[T](encoder: AgnosticEncoder[T], data: Iterator[T]): Dataset[T] = { newDataset(encoder) { builder => if (data.nonEmpty) { -val timeZoneId = conf.get("spark.sql.session.timeZone") val arrowData = ArrowSerializer.serialize(data, encoder, allocator, timeZoneId) if (arrowData.size() <= conf.get("spark.sql.session.localRelationCacheThreshold").toInt) { builder.getLocalRelationBuilder @@ -529,9 +528,11 @@ class SparkSession private[sql] ( client.semanticHash(plan).getSemanticHash.getResult } + private[sql] def timeZoneId: String = conf.get("spark.sql.session.timeZone") + private[sql] def execute[T](plan: proto.Plan, encoder: AgnosticEncoder[T]): SparkResult[T] = { val value = client.execute(plan) -val result = new SparkResult(value, allocator, encoder) +val result = new SparkResult(value, allocator, encoder, timeZoneId) cleaner.register(result) result } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index eed8bd3f37d..e3055b2678f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.util.ArrowUtils private[sql] class SparkResult[T]( responses: java.util.Iterator[proto.ExecutePlanResponse], allocator: BufferAllocator, -encoder: AgnosticEncoder[T]) +encoder: AgnosticEncoder[T], +timeZoneId: String) extends AutoCloseable with Cleanable { self => @@ -213,7 +214,8 @@ private[sql] class SparkResult[T]( new ConcatenatingArrowStreamReader( allocator, Iterator.single(new ResultMessageIterator(destructive)), - destructive)) + destructive), +timeZoneId) } } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowD
[spark] branch master updated: [SPARK-44449][CONNECT] Upcasting for direct Arrow Deserialization
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a6ac63d14b5 [SPARK-9][CONNECT] Upcasting for direct Arrow Deserialization a6ac63d14b5 is described below commit a6ac63d14b56d939dda1aa2a8e74308efc8e1b93 Author: Herman van Hovell AuthorDate: Mon Jul 24 20:46:01 2023 -0400 [SPARK-9][CONNECT] Upcasting for direct Arrow Deserialization ### What changes were proposed in this pull request? This PR adds upcasting to direct Arrow deserialization for the Spark Connect Scala Client. This is implemented by decoupling leaf encoders from vector implementations, instead all leaf encoders are now tied to an `ArrowVectorReader` instance that will allow us to read the data we need from an arbitrary vector type, provided we can read this data without data loss (this is both checked at runtime and compile time). ### Why are the changes needed? Direct Arrow deserialization did not support upcasting yet. Not supporting this would be a regression for connect compared to SPARK 3.4. ### Does this PR introduce _any_ user-facing change? Yes, it brings arrow encoders up to par with the existing catalyst encoding framework. ### How was this patch tested? Added tests to `ArrowEncoderSuite`. Re-enabled tests that relied on upcasting. Closes #42076 from hvanhovell/SPARK-9. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 5 +- .../spark/sql/connect/client/SparkResult.scala | 6 +- .../connect/client/arrow/ArrowDeserializer.scala | 221 ++ .../connect/client/arrow/ArrowVectorReader.scala | 259 + .../org/apache/spark/sql/ClientE2ETestSuite.scala | 10 +- .../sql/KeyValueGroupedDatasetE2ETestSuite.scala | 5 +- .../connect/client/arrow/ArrowEncoderSuite.scala | 217 +++-- 7 files changed, 595 insertions(+), 128 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index b37e3884038..161b5a0217e 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -126,7 +126,6 @@ class SparkSession private[sql] ( private def createDataset[T](encoder: AgnosticEncoder[T], data: Iterator[T]): Dataset[T] = { newDataset(encoder) { builder => if (data.nonEmpty) { -val timeZoneId = conf.get("spark.sql.session.timeZone") val arrowData = ArrowSerializer.serialize(data, encoder, allocator, timeZoneId) if (arrowData.size() <= conf.get("spark.sql.session.localRelationCacheThreshold").toInt) { builder.getLocalRelationBuilder @@ -529,9 +528,11 @@ class SparkSession private[sql] ( client.semanticHash(plan).getSemanticHash.getResult } + private[sql] def timeZoneId: String = conf.get("spark.sql.session.timeZone") + private[sql] def execute[T](plan: proto.Plan, encoder: AgnosticEncoder[T]): SparkResult[T] = { val value = client.execute(plan) -val result = new SparkResult(value, allocator, encoder) +val result = new SparkResult(value, allocator, encoder, timeZoneId) cleaner.register(result) result } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index eed8bd3f37d..e3055b2678f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.util.ArrowUtils private[sql] class SparkResult[T]( responses: java.util.Iterator[proto.ExecutePlanResponse], allocator: BufferAllocator, -encoder: AgnosticEncoder[T]) +encoder: AgnosticEncoder[T], +timeZoneId: String) extends AutoCloseable with Cleanable { self => @@ -213,7 +214,8 @@ private[sql] class SparkResult[T]( new ConcatenatingArrowStreamReader( allocator, Iterator.single(new ResultMessageIterator(destructive)), - destructive)) + destructive), +timeZoneId) } } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializ
[spark] branch branch-3.5 updated: [SPARK-44496][SQL][CONNECT] Move Interfaces needed by SCSC to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new b9d6b9a2658 [SPARK-44496][SQL][CONNECT] Move Interfaces needed by SCSC to sql/api b9d6b9a2658 is described below commit b9d6b9a26589100db682bb6b6d66eb0fb49df85e Author: Rui Wang AuthorDate: Sat Jul 22 19:53:04 2023 -0400 [SPARK-44496][SQL][CONNECT] Move Interfaces needed by SCSC to sql/api ### What changes were proposed in this pull request? This PR moves some interfaces and utils that are needed by scala client to sql/api. ### Why are the changes needed? So that scala client does not need to depend on Spark. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT Closes #42092 from amaliujia/row_coder. Authored-by: Rui Wang Signed-off-by: Herman van Hovell (cherry picked from commit 515dfc166a95ecc8decce0f0cd99e06fe395f94f) Signed-off-by: Herman van Hovell --- .../spark/api/java/function/CoGroupFunction.java | 0 .../api/java/function/DoubleFlatMapFunction.java | 0 .../spark/api/java/function/DoubleFunction.java| 0 .../spark/api/java/function/FilterFunction.java| 0 .../spark/api/java/function/FlatMapFunction.java | 0 .../spark/api/java/function/FlatMapFunction2.java | 0 .../api/java/function/FlatMapGroupsFunction.java | 0 .../spark/api/java/function/ForeachFunction.java | 0 .../java/function/ForeachPartitionFunction.java| 0 .../apache/spark/api/java/function/Function.java | 0 .../apache/spark/api/java/function/Function0.java | 0 .../apache/spark/api/java/function/Function2.java | 0 .../apache/spark/api/java/function/Function3.java | 0 .../apache/spark/api/java/function/Function4.java | 0 .../spark/api/java/function/MapFunction.java | 0 .../spark/api/java/function/MapGroupsFunction.java | 0 .../api/java/function/MapPartitionsFunction.java | 0 .../api/java/function/PairFlatMapFunction.java | 0 .../spark/api/java/function/PairFunction.java | 0 .../spark/api/java/function/ReduceFunction.java| 0 .../spark/api/java/function/VoidFunction.java | 0 .../spark/api/java/function/VoidFunction2.java | 0 .../spark/api/java/function/package-info.java | 0 .../apache/spark/api/java/function/package.scala | 0 .../org/apache/spark/util/SparkClassUtils.scala| 4 ++ connector/connect/client/jvm/pom.xml | 5 ++ .../main/scala/org/apache/spark/sql/Column.scala | 4 +- .../org/apache/spark/sql/DataFrameReader.scala | 8 +-- .../main/scala/org/apache/spark/sql/Dataset.scala | 12 ++--- .../main/scala/org/apache/spark/util/Utils.scala | 1 - project/MimaExcludes.scala | 29 +++ .../main/scala/org/apache/spark/sql/Encoder.scala | 0 .../src/main/scala/org/apache/spark/sql/Row.scala | 6 +-- .../scala/org/apache/spark/sql/SqlApiConf.scala| 2 + .../sql/catalyst/encoders/AgnosticEncoder.scala| 4 +- .../sql/catalyst/expressions/GenericRow.scala | 25 ++--- .../sql/catalyst/expressions/OrderUtils.scala | 25 + .../sql/catalyst/util/SparkCharVarcharUtils.scala | 60 ++ .../apache/spark/sql/errors/DataTypeErrors.scala | 28 ++ .../spark/sql/catalyst/expressions/ordering.scala | 9 +--- .../spark/sql/catalyst/expressions/rows.scala | 21 .../spark/sql/catalyst/util/CharVarcharUtils.scala | 38 +- .../spark/sql/errors/QueryCompilationErrors.scala | 8 +-- .../spark/sql/errors/QueryExecutionErrors.scala| 10 +--- .../apache/spark/sql/CharVarcharTestSuite.scala| 28 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../sql/test/DataFrameReaderWriterSuite.scala | 12 ++--- 47 files changed, 203 insertions(+), 140 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java b/common/utils/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java similarity index 100% rename from core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java rename to common/utils/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java b/common/utils/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java similarity index 100% rename from core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java rename to common/utils/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java b/common/utils/src/main/java/org/apache/spark/api/java
[spark] branch master updated: [SPARK-44496][SQL][CONNECT] Move Interfaces needed by SCSC to sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 515dfc166a9 [SPARK-44496][SQL][CONNECT] Move Interfaces needed by SCSC to sql/api 515dfc166a9 is described below commit 515dfc166a95ecc8decce0f0cd99e06fe395f94f Author: Rui Wang AuthorDate: Sat Jul 22 19:53:04 2023 -0400 [SPARK-44496][SQL][CONNECT] Move Interfaces needed by SCSC to sql/api ### What changes were proposed in this pull request? This PR moves some interfaces and utils that are needed by scala client to sql/api. ### Why are the changes needed? So that scala client does not need to depend on Spark. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT Closes #42092 from amaliujia/row_coder. Authored-by: Rui Wang Signed-off-by: Herman van Hovell --- .../spark/api/java/function/CoGroupFunction.java | 0 .../api/java/function/DoubleFlatMapFunction.java | 0 .../spark/api/java/function/DoubleFunction.java| 0 .../spark/api/java/function/FilterFunction.java| 0 .../spark/api/java/function/FlatMapFunction.java | 0 .../spark/api/java/function/FlatMapFunction2.java | 0 .../api/java/function/FlatMapGroupsFunction.java | 0 .../spark/api/java/function/ForeachFunction.java | 0 .../java/function/ForeachPartitionFunction.java| 0 .../apache/spark/api/java/function/Function.java | 0 .../apache/spark/api/java/function/Function0.java | 0 .../apache/spark/api/java/function/Function2.java | 0 .../apache/spark/api/java/function/Function3.java | 0 .../apache/spark/api/java/function/Function4.java | 0 .../spark/api/java/function/MapFunction.java | 0 .../spark/api/java/function/MapGroupsFunction.java | 0 .../api/java/function/MapPartitionsFunction.java | 0 .../api/java/function/PairFlatMapFunction.java | 0 .../spark/api/java/function/PairFunction.java | 0 .../spark/api/java/function/ReduceFunction.java| 0 .../spark/api/java/function/VoidFunction.java | 0 .../spark/api/java/function/VoidFunction2.java | 0 .../spark/api/java/function/package-info.java | 0 .../apache/spark/api/java/function/package.scala | 0 .../org/apache/spark/util/SparkClassUtils.scala| 4 ++ connector/connect/client/jvm/pom.xml | 5 ++ .../main/scala/org/apache/spark/sql/Column.scala | 4 +- .../org/apache/spark/sql/DataFrameReader.scala | 8 +-- .../main/scala/org/apache/spark/sql/Dataset.scala | 12 ++--- .../main/scala/org/apache/spark/util/Utils.scala | 1 - project/MimaExcludes.scala | 29 +++ .../main/scala/org/apache/spark/sql/Encoder.scala | 0 .../src/main/scala/org/apache/spark/sql/Row.scala | 6 +-- .../scala/org/apache/spark/sql/SqlApiConf.scala| 2 + .../sql/catalyst/encoders/AgnosticEncoder.scala| 4 +- .../sql/catalyst/expressions/GenericRow.scala | 25 ++--- .../sql/catalyst/expressions/OrderUtils.scala | 25 + .../sql/catalyst/util/SparkCharVarcharUtils.scala | 60 ++ .../apache/spark/sql/errors/DataTypeErrors.scala | 28 ++ .../spark/sql/catalyst/expressions/ordering.scala | 9 +--- .../spark/sql/catalyst/expressions/rows.scala | 21 .../spark/sql/catalyst/util/CharVarcharUtils.scala | 38 +- .../spark/sql/errors/QueryCompilationErrors.scala | 8 +-- .../spark/sql/errors/QueryExecutionErrors.scala| 10 +--- .../apache/spark/sql/CharVarcharTestSuite.scala| 28 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../sql/test/DataFrameReaderWriterSuite.scala | 12 ++--- 47 files changed, 203 insertions(+), 140 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java b/common/utils/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java similarity index 100% rename from core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java rename to common/utils/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java b/common/utils/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java similarity index 100% rename from core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java rename to common/utils/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java b/common/utils/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java similarity index 100% rename from core/src/main/java/org/apache/spark/api/java/function
[spark] branch branch-3.5 updated: [SPARK-44396][CONNECT] Direct Arrow Deserialization
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 244d02be8e4 [SPARK-44396][CONNECT] Direct Arrow Deserialization 244d02be8e4 is described below commit 244d02be8e4535458d2c12be7b461aa1b1e497f6 Author: Herman van Hovell AuthorDate: Wed Jul 19 09:26:26 2023 -0400 [SPARK-44396][CONNECT] Direct Arrow Deserialization ### What changes were proposed in this pull request? This PR adds direct arrow to user object deserialization to the Spark Connect Scala Client. ### Why are the changes needed? We want to decouple the scala client from catalyst. We need a way to encode user object from and to arrrow. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added tests to `ArrowEncoderSuite`. Closes #42011 from hvanhovell/SPARK-44396. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 5939b75b5fe701cb63fedc64f57c9f0a15ef9202) Signed-off-by: Herman van Hovell --- connector/connect/client/jvm/pom.xml | 19 + .../client/arrow/ScalaCollectionUtils.scala| 38 ++ .../client/arrow/ScalaCollectionUtils.scala| 37 ++ .../spark/sql/connect/client/SparkResult.scala | 230 + .../connect/client/arrow/ArrowDeserializer.scala | 533 + .../connect/client/arrow/ArrowEncoderUtils.scala | 3 + .../arrow/ConcatenatingArrowStreamReader.scala | 185 +++ .../org/apache/spark/sql/ClientE2ETestSuite.scala | 49 +- .../sql/KeyValueGroupedDatasetE2ETestSuite.scala | 36 +- .../spark/sql/application/ReplE2ESuite.scala | 6 +- .../connect/client/arrow/ArrowEncoderSuite.scala | 127 +++-- 11 files changed, 1085 insertions(+), 178 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 8a51bf65d6a..0f6783cbd68 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -140,6 +140,7 @@ + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes @@ -224,6 +225,24 @@ + +org.codehaus.mojo +build-helper-maven-plugin + + +add-sources +generate-sources + + add-source + + + +src/main/scala-${scala.binary.version} + + + + + \ No newline at end of file diff --git a/connector/connect/client/jvm/src/main/scala-2.12/org/apache/spark/sql/connect/client/arrow/ScalaCollectionUtils.scala b/connector/connect/client/jvm/src/main/scala-2.12/org/apache/spark/sql/connect/client/arrow/ScalaCollectionUtils.scala new file mode 100644 index 000..c2e01d974e0 --- /dev/null +++ b/connector/connect/client/jvm/src/main/scala-2.12/org/apache/spark/sql/connect/client/arrow/ScalaCollectionUtils.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connect.client.arrow + +import scala.collection.generic.{GenericCompanion, GenMapFactory} +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.connect.client.arrow.ArrowDeserializers.resolveCompanion + +/** + * A couple of scala version specific collection utility functions. + */ +private[arrow] object ScalaCollectionUtils { + def getIterableCompanion(tag: ClassTag[_]): GenericCompanion[Iterable] = { +ArrowDeserializers.resolveCompanion[GenericCompanion[Iterable]](tag) + } + def getMapCompanion(tag: ClassTag[_]): GenMapFactory[Map] = { +resolveCompanion[GenMapFactory[Map]](tag) + } + def wrap[T](array: AnyRef): mutable.WrappedArray[T] = { +mutable.WrappedArray.make(array) + } +} diff --git a/connector/connect/client/jvm/src/main/scala-2.13/org/apache/spark/sql/connect/client/arrow/ScalaC
[spark] branch master updated: [SPARK-44396][CONNECT] Direct Arrow Deserialization
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5939b75b5fe [SPARK-44396][CONNECT] Direct Arrow Deserialization 5939b75b5fe is described below commit 5939b75b5fe701cb63fedc64f57c9f0a15ef9202 Author: Herman van Hovell AuthorDate: Wed Jul 19 09:26:26 2023 -0400 [SPARK-44396][CONNECT] Direct Arrow Deserialization ### What changes were proposed in this pull request? This PR adds direct arrow to user object deserialization to the Spark Connect Scala Client. ### Why are the changes needed? We want to decouple the scala client from catalyst. We need a way to encode user object from and to arrrow. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added tests to `ArrowEncoderSuite`. Closes #42011 from hvanhovell/SPARK-44396. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- connector/connect/client/jvm/pom.xml | 19 + .../client/arrow/ScalaCollectionUtils.scala| 38 ++ .../client/arrow/ScalaCollectionUtils.scala| 37 ++ .../spark/sql/connect/client/SparkResult.scala | 230 + .../connect/client/arrow/ArrowDeserializer.scala | 533 + .../connect/client/arrow/ArrowEncoderUtils.scala | 3 + .../arrow/ConcatenatingArrowStreamReader.scala | 185 +++ .../org/apache/spark/sql/ClientE2ETestSuite.scala | 49 +- .../sql/KeyValueGroupedDatasetE2ETestSuite.scala | 36 +- .../spark/sql/application/ReplE2ESuite.scala | 6 +- .../connect/client/arrow/ArrowEncoderSuite.scala | 127 +++-- 11 files changed, 1085 insertions(+), 178 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 93cc782ab13..60ed0f3ba46 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -140,6 +140,7 @@ + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes @@ -224,6 +225,24 @@ + +org.codehaus.mojo +build-helper-maven-plugin + + +add-sources +generate-sources + + add-source + + + +src/main/scala-${scala.binary.version} + + + + + \ No newline at end of file diff --git a/connector/connect/client/jvm/src/main/scala-2.12/org/apache/spark/sql/connect/client/arrow/ScalaCollectionUtils.scala b/connector/connect/client/jvm/src/main/scala-2.12/org/apache/spark/sql/connect/client/arrow/ScalaCollectionUtils.scala new file mode 100644 index 000..c2e01d974e0 --- /dev/null +++ b/connector/connect/client/jvm/src/main/scala-2.12/org/apache/spark/sql/connect/client/arrow/ScalaCollectionUtils.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connect.client.arrow + +import scala.collection.generic.{GenericCompanion, GenMapFactory} +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.connect.client.arrow.ArrowDeserializers.resolveCompanion + +/** + * A couple of scala version specific collection utility functions. + */ +private[arrow] object ScalaCollectionUtils { + def getIterableCompanion(tag: ClassTag[_]): GenericCompanion[Iterable] = { +ArrowDeserializers.resolveCompanion[GenericCompanion[Iterable]](tag) + } + def getMapCompanion(tag: ClassTag[_]): GenMapFactory[Map] = { +resolveCompanion[GenMapFactory[Map]](tag) + } + def wrap[T](array: AnyRef): mutable.WrappedArray[T] = { +mutable.WrappedArray.make(array) + } +} diff --git a/connector/connect/client/jvm/src/main/scala-2.13/org/apache/spark/sql/connect/client/arrow/ScalaCollectionUtils.scala b/connector/connect/client/jvm/src/main/scala-2.13/org/apache/spark/sql/
[spark] branch master updated: [SPARK-42981][CONNECT] Add direct arrow serialization
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 947164879b0 [SPARK-42981][CONNECT] Add direct arrow serialization 947164879b0 is described below commit 947164879b01e892ebeedada70e1527b55eb9010 Author: Herman van Hovell AuthorDate: Wed Jul 12 17:24:09 2023 -0400 [SPARK-42981][CONNECT] Add direct arrow serialization ### What changes were proposed in this pull request? This PR adds direct serialization from user domain objects to arrow batches. This removes the need to go through catalyst. ### Why are the changes needed? We want to minimalize the number of dependencies in connect. Removing catalyst and core is part of this effort. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? I added the `ArrowEncoderSuite` to test this. Closes #40611 from hvanhovell/SPARK-42981. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 8 +- .../spark/sql/connect/client/SparkResult.scala | 1 - .../connect/client/arrow/ArrowEncoderUtils.scala | 48 ++ .../sql/connect/client/arrow/ArrowSerializer.scala | 546 + .../sql/connect/client/util/ConvertToArrow.scala | 74 -- .../connect/client/arrow/ArrowEncoderSuite.scala | 894 + 6 files changed, 1492 insertions(+), 79 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 529ba97c40d..c27f0f32e0d 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -38,7 +38,8 @@ import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BoxedLongEncoder, UnboundRowEncoder} import org.apache.spark.sql.connect.client.{ClassFinder, SparkConnectClient, SparkResult} import org.apache.spark.sql.connect.client.SparkConnectClient.Configuration -import org.apache.spark.sql.connect.client.util.{Cleaner, ConvertToArrow} +import org.apache.spark.sql.connect.client.arrow.ArrowSerializer +import org.apache.spark.sql.connect.client.util.Cleaner import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.toLiteralProto import org.apache.spark.sql.internal.CatalogImpl import org.apache.spark.sql.streaming.DataStreamReader @@ -126,9 +127,8 @@ class SparkSession private[sql] ( newDataset(encoder) { builder => if (data.nonEmpty) { val timeZoneId = conf.get("spark.sql.session.timeZone") -val (arrowData, arrowDataSize) = - ConvertToArrow(encoder, data, timeZoneId, errorOnDuplicatedFieldNames = true, allocator) -if (arrowDataSize <= conf.get("spark.sql.session.localRelationCacheThreshold").toInt) { +val arrowData = ArrowSerializer.serialize(data, encoder, allocator, timeZoneId) +if (arrowData.size() <= conf.get("spark.sql.session.localRelationCacheThreshold").toInt) { builder.getLocalRelationBuilder .setSchema(encoder.schema.json) .setData(arrowData) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index d33f405ee94..c0ad026e764 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -101,7 +101,6 @@ private[sql] class SparkResult[T]( } while (reader.loadNextBatch()) { val rowCount = root.getRowCount -assert(root.getRowCount == response.getArrowBatch.getRowCount) // HUH! if (rowCount > 0) { val vectors = root.getFieldVectors.asScala .map(v => new ArrowColumnVector(transferToNewVector(v))) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderUtils.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderUtils.scala new file mode 100644 index 000..f6b140bae55 --- /dev/null +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderUtils.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the N
[spark] branch master updated: [SPARK-44353][CONNECT][SQL] Remove StructType.toAttributes
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6161bf44f40 [SPARK-44353][CONNECT][SQL] Remove StructType.toAttributes 6161bf44f40 is described below commit 6161bf44f40f8146ea4c115c788fd4eaeb128769 Author: Herman van Hovell AuthorDate: Wed Jul 12 02:27:16 2023 -0400 [SPARK-44353][CONNECT][SQL] Remove StructType.toAttributes ### What changes were proposed in this pull request? This PR removes StructType.toAttribute. It is being replace by DataTypeUtils.toAttribute(..). ### Why are the changes needed? We want to move the DataType hierarchy into sql/api. We need to remove any catalyst specific API. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #41925 from hvanhovell/SPARK-44353. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../spark/sql/connect/client/SparkResult.scala | 4 +- .../sql/connect/planner/SparkConnectPlanner.scala | 18 +++--- .../connect/planner/SparkConnectPlannerSuite.scala | 19 +- .../connect/planner/SparkConnectProtoSuite.scala | 12 ++-- .../spark/sql/kafka010/KafkaBatchWrite.scala | 5 +- .../spark/sql/kafka010/KafkaStreamingWrite.scala | 5 +- .../spark/sql/catalyst/analysis/Analyzer.scala | 7 ++- .../sql/catalyst/analysis/AssignmentUtils.scala| 3 +- .../catalyst/analysis/ResolveInlineTables.scala| 2 +- .../catalyst/analysis/TableOutputResolver.scala| 5 +- .../sql/catalyst/analysis/v2ResolutionPlans.scala | 3 +- .../sql/catalyst/encoders/ExpressionEncoder.scala | 3 +- .../catalyst/optimizer/NestedColumnAliasing.scala | 4 +- .../spark/sql/catalyst/optimizer/Optimizer.scala | 3 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 3 +- .../sql/catalyst/plans/logical/LocalRelation.scala | 7 ++- .../plans/logical/basicLogicalOperators.scala | 3 +- .../spark/sql/catalyst/plans/logical/object.scala | 7 ++- .../spark/sql/catalyst/types/DataTypeUtils.scala | 7 +++ .../spark/sql/catalyst/util/GeneratedColumn.scala | 4 +- .../sql/connector/catalog/CatalogV2Implicits.scala | 3 +- .../datasources/v2/DataSourceV2Implicits.scala | 3 +- .../datasources/v2/DataSourceV2Relation.scala | 3 +- .../org/apache/spark/sql/types/StructType.scala| 6 +- .../scala/org/apache/spark/sql/HashBenchmark.scala | 3 +- .../spark/sql/UnsafeProjectionBenchmark.scala | 9 +-- .../sql/catalyst/analysis/AnalysisSuite.scala | 13 ++-- .../CreateTablePartitioningValidationSuite.scala | 3 +- .../catalyst/analysis/V2WriteAnalysisSuite.scala | 72 -- .../catalyst/encoders/EncoderResolutionSuite.scala | 5 +- .../catalyst/encoders/ExpressionEncoderSuite.scala | 3 +- .../catalyst/expressions/SelectedFieldSuite.scala | 4 +- .../main/scala/org/apache/spark/sql/Dataset.scala | 5 +- .../spark/sql/RelationalGroupedDataset.scala | 7 ++- .../scala/org/apache/spark/sql/SparkSession.scala | 11 ++-- .../execution/aggregate/HashAggregateExec.scala| 5 +- .../spark/sql/execution/aggregate/udaf.scala | 5 +- .../sql/execution/arrow/ArrowConverters.scala | 3 +- .../spark/sql/execution/command/SetCommand.scala | 5 +- .../apache/spark/sql/execution/command/ddl.scala | 5 +- .../spark/sql/execution/command/functions.scala| 3 +- .../sql/execution/datasources/FileFormat.scala | 3 +- .../execution/datasources/LogicalRelation.scala| 5 +- .../datasources/SaveIntoDataSourceCommand.scala| 3 +- .../sql/execution/datasources/SchemaPruning.scala | 4 +- .../execution/datasources/orc/OrcFileFormat.scala | 3 +- .../datasources/parquet/ParquetFileFormat.scala| 3 +- .../spark/sql/execution/datasources/rules.scala| 3 +- .../sql/execution/datasources/v2/FileScan.scala| 7 ++- .../sql/execution/datasources/v2/FileWrite.scala | 3 +- .../v2/PartitionReaderWithPartitionValues.scala| 3 +- .../execution/datasources/v2/PushDownUtils.scala | 3 +- .../datasources/v2/V2ScanRelationPushDown.scala| 3 +- .../execution/streaming/StreamingRelation.scala| 7 ++- .../spark/sql/execution/streaming/memory.scala | 3 +- .../streaming/sources/ConsoleStreamingWrite.scala | 3 +- .../streaming/sources/ForeachWriterTable.scala | 3 +- .../state/FlatMapGroupsWithStateExecHelper.scala | 5 +- .../state/SymmetricHashJoinStateManager.scala | 3 +- .../apache/spark/sql/internal/CatalogImpl.scala| 3 +- .../spark/sql/streaming/DataStreamReader.scala | 3 +- .../spark/sql/streaming/DataStreamWriter.scala | 4 +- .../sql/connector/TableCapabilityCheckSuite.scala | 5 +- .../connector/V2CommandsCaseSensitivitySuite.scala | 3
[spark] branch master updated (f6866d1c1ab -> 453300b418b)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from f6866d1c1ab [SPARK-44343][CONNECT] Prepare ScalaReflection to the move to SQL/API add 453300b418b [SPARK-44352][CONNECT] Put back sameType and friends in DataType No new revisions were added by this update. Summary of changes: .../spark/sql/catalyst/types/DataTypeUtils.scala | 40 ++- .../org/apache/spark/sql/types/DataType.scala | 59 -- .../org/apache/spark/sql/types/StructType.scala| 2 +- 3 files changed, 60 insertions(+), 41 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44343][CONNECT] Prepare ScalaReflection to the move to SQL/API
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f6866d1c1ab [SPARK-44343][CONNECT] Prepare ScalaReflection to the move to SQL/API f6866d1c1ab is described below commit f6866d1c1ab986545f2b3c1fb254d6ca0d56c056 Author: Herman van Hovell AuthorDate: Mon Jul 10 14:45:04 2023 -0400 [SPARK-44343][CONNECT] Prepare ScalaReflection to the move to SQL/API ### What changes were proposed in this pull request? This PR moves all catalyst specific internals out of ScalaReflection into other catalyst classes: - Serializer Expression Generation is moved to `SerializerBuildHelper`. - Deaerializer Expression Generation is moved to `DeserializerBuildHelper`. - Common utils are moved to `EncoderUtils`. ### Why are the changes needed? We want to use ScalaReflection based encoder inference both for SQL/Core and Connect. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #41920 from hvanhovell/SPARK-44343. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../sql/catalyst/DeserializerBuildHelper.scala | 254 - .../spark/sql/catalyst/ScalaReflection.scala | 573 + .../spark/sql/catalyst/SerializerBuildHelper.scala | 199 ++- .../spark/sql/catalyst/encoders/EncoderUtils.scala | 139 + .../sql/catalyst/encoders/ExpressionEncoder.scala | 6 +- .../catalyst/expressions/V2ExpressionUtils.scala | 5 +- .../sql/catalyst/expressions/objects/objects.scala | 20 +- .../spark/sql/catalyst/ScalaReflectionSuite.scala | 4 +- .../expressions/ObjectExpressionsSuite.scala | 6 +- .../datasources/parquet/ParquetIOSuite.scala | 4 +- .../datasources/parquet/ParquetSchemaSuite.scala | 18 +- .../datasources/parquet/ParquetTest.scala | 3 + .../apache/spark/sql/internal/CatalogSuite.scala | 16 +- 13 files changed, 630 insertions(+), 617 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala index 41fd5bb239d..bdf996424ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala @@ -17,10 +17,14 @@ package org.apache.spark.sql.catalyst -import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue -import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField, UpCast} -import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, Invoke, StaticInvoke} -import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils} +import org.apache.spark.sql.catalyst.{expressions => exprs} +import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedExtractValue} +import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BoxedLeafEncoder, DateEncoder, DayTimeIntervalEncoder, InstantEncoder, IterableEncoder, JavaBeanEncoder, JavaBigIntEncoder, JavaDecimalEncoder, JavaEnumEncoder, LocalDateEncoder, LocalDateTimeEncoder, MapEncoder, OptionEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, ProductEncoder, ScalaBigIn [...] +import org.apache.spark.sql.catalyst.encoders.EncoderUtils.{externalDataTypeFor, isNativeEncoder} +import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField, IsNull, MapKeys, MapValues, UpCast} +import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, CreateExternalRow, InitializeJavaBean, Invoke, NewInstance, StaticInvoke, UnresolvedCatalystToExternalMap, UnresolvedMapObjects, WrapOption} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, IntervalUtils} import org.apache.spark.sql.types._ object DeserializerBuildHelper { @@ -193,4 +197,246 @@ object DeserializerBuildHelper { UpCast(expr, DecimalType, walkedTypePath.getPaths) case _ => UpCast(expr, expected, walkedTypePath.getPaths) } + + /** + * Returns an expression for deserializing the Spark SQL representation of an object into its + * external form. The mapping between the internal and external representations is + * described by encoder `enc`. The Spark SQL representation is located at ordinal 0 of + * a row, i.e., `GetColumnByOrdinal(0, _)`. Nested classes will have their fields accessed using + * `UnresolvedExtractValue`. + * + * The re
[spark] branch master updated: [SPARK-44271][SQL] Move default values functions from StructType to ResolveDefaultColumns
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c37d7dec8aa [SPARK-44271][SQL] Move default values functions from StructType to ResolveDefaultColumns c37d7dec8aa is described below commit c37d7dec8aa4d703b6dac9b9d60ff25d9d5dc665 Author: Rui Wang AuthorDate: Mon Jul 10 06:23:03 2023 -0400 [SPARK-44271][SQL] Move default values functions from StructType to ResolveDefaultColumns ### What changes were proposed in this pull request? Move default values functions from StructType to ResolveDefaultColumns. ### Why are the changes needed? To simply DataType interface. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing test Closes #41820 from amaliujia/clean_up_left_errors. Authored-by: Rui Wang Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/csv/UnivocityParser.scala | 2 +- .../spark/sql/catalyst/json/JacksonParser.scala| 7 +++-- .../catalyst/util/ResolveDefaultColumnsUtil.scala | 32 -- .../org/apache/spark/sql/types/StructType.scala| 8 -- .../apache/spark/sql/types/StructTypeSuite.scala | 16 +-- .../datasources/orc/OrcColumnarBatchReader.java| 3 +- .../parquet/VectorizedParquetRecordReader.java | 3 +- .../datasources/orc/OrcDeserializer.scala | 15 ++ .../datasources/parquet/ParquetRowConverter.scala | 15 ++ 9 files changed, 58 insertions(+), 43 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index b58649da61c..a02d57c0bc7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -353,7 +353,7 @@ class UnivocityParser( case NonFatal(e) => badRecordException = badRecordException.orElse(Some(e)) // Use the corresponding DEFAULT value associated with the column, if any. - row.update(i, requiredSchema.existenceDefaultValues(i)) + row.update(i, ResolveDefaultColumns.existenceDefaultValues(requiredSchema)(i)) } i += 1 } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 5286e16b088..03dce431837 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -448,14 +448,15 @@ class JacksonParser( var skipRow = false structFilters.reset() -resetExistenceDefaultsBitmask(schema) +lazy val bitmask = ResolveDefaultColumns.existenceDefaultsBitmask(schema) +resetExistenceDefaultsBitmask(schema, bitmask) while (!skipRow && nextUntil(parser, JsonToken.END_OBJECT)) { schema.getFieldIndex(parser.getCurrentName) match { case Some(index) => try { row.update(index, fieldConverters(index).apply(parser)) skipRow = structFilters.skipRow(row, index) -schema.existenceDefaultsBitmask(index) = false +bitmask(index) = false } catch { case e: SparkUpgradeException => throw e case NonFatal(e) if isRoot || enablePartialResults => @@ -469,7 +470,7 @@ class JacksonParser( if (skipRow) { None } else if (badRecordException.isEmpty) { - applyExistenceDefaultValuesToRow(schema, row) + applyExistenceDefaultValuesToRow(schema, row, bitmask) Some(row) } else { throw PartialResultException(row, badRecordException.get) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala index 26efa8c8df2..6489fb9aaaf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala @@ -396,27 +396,30 @@ object ResolveDefaultColumns extends QueryErrorsBase { * above, for convenience. */ def getExistenceDefaultsBitmask(schema: StructType): Array[Boolean] = { -Array.fill[Boolean](schema.existenceDefaultValues.size)(true) +Array.fill[Boolean](existenceDefaultValues(schema).size)(true) } /** * Resets the elements of the array initially returned from [[getExistenceDef
[spark] branch master updated: [SPARK-44321][CONNECT] Decouple ParseException from AnalysisException
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new fdeb8d8551e [SPARK-44321][CONNECT] Decouple ParseException from AnalysisException fdeb8d8551e is described below commit fdeb8d8551e3389e734a8416c63f942c6e20b063 Author: Herman van Hovell AuthorDate: Sat Jul 8 15:10:44 2023 -0400 [SPARK-44321][CONNECT] Decouple ParseException from AnalysisException ### What changes were proposed in this pull request? This PR decouples ParseException from AnalysisException. ### Why are the changes needed? We are moving (parts of) parsing to sql/api to share (datatype) parsing between connect and sql. ### Does this PR introduce _any_ user-facing change? Yes and no. This is a breaking change in the sense that the parent class of the ParseException changes from AnalysisException to SparkException. ### How was this patch tested? Existing tests. Closes #41879 from hvanhovell/SPARK-44321. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- docs/sql-migration-guide.md| 1 + .../apache/spark/sql/catalyst/trees/origin.scala | 7 +++ .../org/apache/spark/sql/AnalysisException.scala | 6 ++- .../apache/spark/sql/catalyst/parser/parsers.scala | 51 ++ .../apache/spark/sql/catalyst/trees/TreeNode.scala | 7 ++- .../analyzer-results/csv-functions.sql.out | 2 +- .../analyzer-results/json-functions.sql.out| 2 +- .../sql-tests/results/csv-functions.sql.out| 2 +- .../sql-tests/results/json-functions.sql.out | 2 +- .../org/apache/spark/sql/JsonFunctionsSuite.scala | 3 +- .../apache/spark/sql/LateralColumnAliasSuite.scala | 3 +- .../org/apache/spark/sql/ParametersSuite.scala | 13 +++--- .../org/apache/spark/sql/SQLInsertTestSuite.scala | 3 +- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 5 ++- .../spark/sql/connector/AlterTableTests.scala | 5 ++- .../apache/spark/sql/execution/SQLViewSuite.scala | 6 +-- .../spark/sql/execution/command/DDLSuite.scala | 23 +- .../execution/command/PlanResolutionSuite.scala| 6 +-- .../command/v2/AlterTableReplaceColumnsSuite.scala | 4 +- .../sql/sources/CreateTableAsSelectSuite.scala | 5 +-- .../org/apache/spark/sql/sources/InsertSuite.scala | 4 +- .../org/apache/spark/sql/hive/InsertSuite.scala| 4 +- .../scala/org/apache/spark/sql/hive/UDFSuite.scala | 5 ++- .../spark/sql/hive/execution/HiveDDLSuite.scala| 12 ++--- .../spark/sql/hive/execution/SQLQuerySuite.scala | 6 +-- 25 files changed, 110 insertions(+), 77 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index b1452d41328..42c36ba6818 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -28,6 +28,7 @@ license: | - Since Spark 3.5, Spark thrift server will interrupt task when canceling a running statement. To restore the previous behavior, set `spark.sql.thriftServer.interruptOnCancel` to `false`. - Since Spark 3.5, the Avro will throw `AnalysisException` when reading Interval types as Date or Timestamp types, or reading Decimal types with lower precision. To restore the legacy behavior, set `spark.sql.legacy.avro.allowIncompatibleSchema` to `true` - Since Spark 3.5, Row's json and prettyJson methods are moved to `ToJsonUtil`. +- Since Spark 3.5, ParseException is a subclass of SparkException instead of AnalysisException. ## Upgrading from Spark SQL 3.3 to 3.4 diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala index ccf131546de..ec3e627ac95 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala @@ -42,6 +42,13 @@ case class Origin( } } +/** + * Helper trait for objects that can be traced back to an [[Origin]]. + */ +trait WithOrigin { + def origin: Origin +} + /** * Provides a location for TreeNodes to ask about the context of their origin. For example, which * line of code is currently being parsed. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index de9b2fa0087..ccc3b5f85f7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.{QueryContext, SparkThrowable, SparkThrowableHelper} import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.plans.logical.Logica
[spark] branch master updated: [SPARK-44326][SQL][CONNECT] Move utils that are used from Scala client to the common modules
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7bfbeb62cb1 [SPARK-44326][SQL][CONNECT] Move utils that are used from Scala client to the common modules 7bfbeb62cb1 is described below commit 7bfbeb62cb1dc58d81243d22888faa688bad8064 Author: Rui Wang AuthorDate: Fri Jul 7 13:38:39 2023 -0400 [SPARK-44326][SQL][CONNECT] Move utils that are used from Scala client to the common modules ### What changes were proposed in this pull request? There are some utils are used in the scala client including ser/derse, datetime and interval utils. These can be moved to the common modules. ### Why are the changes needed? To make sure Scala client does not depend on the Catalyst in the future. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing test Closes #41885 from amaliujia/SPARK-44326. Authored-by: Rui Wang Signed-off-by: Herman van Hovell --- .../org/apache/spark/util/SparkSerDerseUtils.scala | 30 +++ .../spark/sql/execution/streaming/Triggers.scala | 6 +- .../sql/expressions/UserDefinedFunction.scala | 5 +- .../spark/sql/streaming/DataStreamWriter.scala | 4 +- .../main/scala/org/apache/spark/util/Utils.scala | 6 +- .../sql/catalyst/util/SparkDateTimeUtils.scala | 39 +++ .../sql/catalyst/util/SparkIntervalUtils.scala | 263 + .../spark/sql/catalyst/util/DateTimeUtils.scala| 2 +- .../spark/sql/catalyst/util/IntervalUtils.scala| 225 +- 9 files changed, 345 insertions(+), 235 deletions(-) diff --git a/common/utils/src/main/scala/org/apache/spark/util/SparkSerDerseUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/SparkSerDerseUtils.scala new file mode 100644 index 000..e9150618476 --- /dev/null +++ b/common/utils/src/main/scala/org/apache/spark/util/SparkSerDerseUtils.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.util + +import java.io.{ByteArrayOutputStream, ObjectOutputStream} + +object SparkSerDerseUtils { + /** Serialize an object using Java serialization */ + def serialize[T](o: T): Array[Byte] = { +val bos = new ByteArrayOutputStream() +val oos = new ObjectOutputStream(bos) +oos.writeObject(o) +oos.close() +bos.toByteArray + } +} diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index be1b0e8ac0c..ad19ad17805 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -22,8 +22,8 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration.Duration import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY -import org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToMillis -import org.apache.spark.sql.catalyst.util.IntervalUtils +import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.microsToMillis +import org.apache.spark.sql.catalyst.util.SparkIntervalUtils import org.apache.spark.sql.streaming.Trigger import org.apache.spark.unsafe.types.UTF8String @@ -35,7 +35,7 @@ private object Triggers { } def convert(interval: String): Long = { -val cal = IntervalUtils.stringToInterval(UTF8String.fromString(interval)) +val cal = SparkIntervalUtils.stringToInterval(UTF8String.fromString(interval)) if (cal.months != 0) { throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressi
[spark] branch master updated: [SPARK-44333][CONNECT][SQL] Move EnhancedLogicalPlan out of ParserUtils
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f7d47e83ce5 [SPARK-44333][CONNECT][SQL] Move EnhancedLogicalPlan out of ParserUtils f7d47e83ce5 is described below commit f7d47e83ce524b88939c3d33168c199a136dabb6 Author: Herman van Hovell AuthorDate: Fri Jul 7 13:36:58 2023 -0400 [SPARK-44333][CONNECT][SQL] Move EnhancedLogicalPlan out of ParserUtils ### What changes were proposed in this pull request? This PR moves EnhancedLogicalPlan out of ParserUtils. This will allow us to move ParserUtils to sql/api. ### Why are the changes needed? We want to move (data type) parsing to sql/api. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #41890 from hvanhovell/SPARK-44333. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/parser/ParserUtils.scala| 28 - .../apache/spark/sql/catalyst/parser/package.scala | 48 ++ 2 files changed, 48 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index d7a47eb3f37..bb40aca9129 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -25,7 +25,6 @@ import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.misc.Interval import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode, TerminalNodeImpl} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin} import org.apache.spark.sql.errors.QueryParsingErrors @@ -232,33 +231,6 @@ object ParserUtils { /** the column name pattern in quoted regex with qualifier */ val qualifiedEscapedIdentifier = ("((?s).+)" + """.""" + "`((?s).+)`").r - /** Some syntactic sugar which makes it easier to work with optional clauses for LogicalPlans. */ - implicit class EnhancedLogicalPlan(val plan: LogicalPlan) extends AnyVal { -/** - * Create a plan using the block of code when the given context exists. Otherwise return the - * original plan. - */ -def optional(ctx: AnyRef)(f: => LogicalPlan): LogicalPlan = { - if (ctx != null) { -f - } else { -plan - } -} - -/** - * Map a [[LogicalPlan]] to another [[LogicalPlan]] if the passed context exists using the - * passed function. The original plan is returned when the context does not exist. - */ -def optionalMap[C](ctx: C)(f: (C, LogicalPlan) => LogicalPlan): LogicalPlan = { - if (ctx != null) { -f(ctx, plan) - } else { -plan - } -} - } - /** * Normalizes the expression parser tree to a SQL string which will be used to generate * the expression alias. In particular, it concatenates terminal nodes of the tree and diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/package.scala new file mode 100644 index 000..ca78553c270 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/package.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +package object parser { + /** Some syntactic sugar which makes it easier to work with optional clauses for LogicalPlans. */ + implicit class EnhancedLogicalPlan(val plan: LogicalPlan) extends AnyVal { +/** + * Create a plan using the block of code when the given context exists. Otherwise return the +
[spark] branch master updated: [SPARK-44322][CONNECT] Make parser use SqlApiConf instead of SQLConf
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d73bb9d51c9 [SPARK-44322][CONNECT] Make parser use SqlApiConf instead of SQLConf d73bb9d51c9 is described below commit d73bb9d51c991483e673818d6c5e25c6fadd676b Author: Herman van Hovell AuthorDate: Fri Jul 7 13:36:09 2023 -0400 [SPARK-44322][CONNECT] Make parser use SqlApiConf instead of SQLConf ### What changes were proposed in this pull request? This makes the AbstractParser use SqlApiConf instead of SQLConf, and it moves the appropriate configurations. ### Why are the changes needed? We are moving (datatype) parsing into the sql/api package. This does not use SQLConf, but uses SqlApiConf instead. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Existing tests. Closes #41883 from hvanhovell/SPARK-44322. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- sql/api/src/main/scala/org/apache/spark/sql/SqlApiConf.scala | 8 .../scala/org/apache/spark/sql/catalyst/parser/parsers.scala | 7 --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 10 +- 3 files changed, 17 insertions(+), 8 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/SqlApiConf.scala index 07943683ab6..3a074c66614 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/SqlApiConf.scala @@ -32,6 +32,10 @@ private[sql] trait SqlApiConf { def ansiEnabled: Boolean def caseSensitiveAnalysis: Boolean def maxToStringFields: Int + def setOpsPrecedenceEnforced: Boolean + def exponentLiteralAsDecimalEnabled: Boolean + def enforceReservedKeywords: Boolean + def doubleQuotedIdentifiers: Boolean } private[sql] object SqlApiConf { @@ -60,4 +64,8 @@ private[sql] object DefaultSqlApiConf extends SqlApiConf { override def ansiEnabled: Boolean = false override def caseSensitiveAnalysis: Boolean = false override def maxToStringFields: Int = 50 + override def setOpsPrecedenceEnforced: Boolean = false + override def exponentLiteralAsDecimalEnabled: Boolean = false + override def enforceReservedKeywords: Boolean = false + override def doubleQuotedIdentifiers: Boolean = false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala index 27670544e1a..fc7a7cfe1f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala @@ -23,8 +23,7 @@ import org.antlr.v4.runtime.tree.TerminalNodeImpl import org.apache.spark.{QueryContext, SparkThrowableHelper} import org.apache.spark.internal.Logging -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.{AnalysisException, SqlApiConf} import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin} import org.apache.spark.sql.errors.QueryParsingErrors import org.apache.spark.sql.types.{DataType, StructType} @@ -32,7 +31,7 @@ import org.apache.spark.sql.types.{DataType, StructType} /** * Base SQL parsing infrastructure. */ -abstract class AbstractParser extends DataTypeParserInterface with SQLConfHelper with Logging { +abstract class AbstractParser extends DataTypeParserInterface with Logging { /** Creates/Resolves DataType for a given SQL string. */ override def parseDataType(sqlText: String): DataType = parse(sqlText) { parser => astBuilder.visitSingleDataType(parser.singleDataType()) @@ -99,6 +98,8 @@ abstract class AbstractParser extends DataTypeParserInterface with SQLConfHelper e.errorClass, e.messageParameters) } } + + private def conf: SqlApiConf = SqlApiConf.get } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4d7322c6e53..9093b0745a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -177,7 +177,6 @@ object SQLConf { * See [[get]] for more information. */ def setSQLConfGetter(getter: () => SQLConf): Unit = { -SqlApiConf.setConfGetter(getter) confGetter.set(getter) } @@ -5019,9 +5018,9 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def useNullsForMissingDefaultColumnValues: Boolean = getConf(SQLConf.USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES)
[spark] branch master updated: [SPARK-43321][CONNECT] Dataset#Joinwith
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9c7978d80b8 [SPARK-43321][CONNECT] Dataset#Joinwith 9c7978d80b8 is described below commit 9c7978d80b8a95bd7fcc26769eea581849000862 Author: Zhen Li AuthorDate: Thu Jul 6 17:42:53 2023 -0400 [SPARK-43321][CONNECT] Dataset#Joinwith ### What changes were proposed in this pull request? Impl missing method JoinWith with Join relation operation The JoinWith adds `left` and `right` struct type info in the Join relation proto. ### Why are the changes needed? Missing Dataset API ### Does this PR introduce _any_ user-facing change? Yes. Added the missing Dataset#JoinWith method ### How was this patch tested? E2E tests. Closes #40997 from zhenlineo/joinwith. Authored-by: Zhen Li Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 85 +++- .../spark/sql/connect/client/SparkResult.scala | 34 +++- .../org/apache/spark/sql/ClientE2ETestSuite.scala | 193 ++ .../CheckConnectJvmClientCompatibility.scala | 1 - .../main/protobuf/spark/connect/relations.proto| 10 + .../sql/connect/planner/SparkConnectPlanner.scala | 24 ++- python/pyspark/sql/connect/proto/relations_pb2.py | 221 +++-- python/pyspark/sql/connect/proto/relations_pb2.pyi | 48 - .../sql/catalyst/encoders/AgnosticEncoder.scala| 44 ++-- .../spark/sql/catalyst/plans/logical/object.scala | 104 +- .../spark/sql/errors/QueryCompilationErrors.scala | 11 +- .../main/scala/org/apache/spark/sql/Dataset.scala | 104 +- 12 files changed, 639 insertions(+), 240 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 2ea3169486b..4fa5c0b9641 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -20,6 +20,7 @@ import java.util.{Collections, Locale} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.reflect.ClassTag import scala.util.control.NonFatal import org.apache.spark.SparkException @@ -568,7 +569,7 @@ class Dataset[T] private[sql] ( } } - private def toJoinType(name: String): proto.Join.JoinType = { + private def toJoinType(name: String, skipSemiAnti: Boolean = false): proto.Join.JoinType = { name.trim.toLowerCase(Locale.ROOT) match { case "inner" => proto.Join.JoinType.JOIN_TYPE_INNER @@ -580,12 +581,12 @@ class Dataset[T] private[sql] ( proto.Join.JoinType.JOIN_TYPE_LEFT_OUTER case "right" | "rightouter" | "right_outer" => proto.Join.JoinType.JOIN_TYPE_RIGHT_OUTER - case "semi" | "leftsemi" | "left_semi" => + case "semi" | "leftsemi" | "left_semi" if !skipSemiAnti => proto.Join.JoinType.JOIN_TYPE_LEFT_SEMI - case "anti" | "leftanti" | "left_anti" => + case "anti" | "leftanti" | "left_anti" if !skipSemiAnti => proto.Join.JoinType.JOIN_TYPE_LEFT_ANTI - case _ => -throw new IllegalArgumentException(s"Unsupported join type `joinType`.") + case e => +throw new IllegalArgumentException(s"Unsupported join type '$e'.") } } @@ -835,6 +836,80 @@ class Dataset[T] private[sql] ( } } + /** + * Joins this Dataset returning a `Tuple2` for each pair where `condition` evaluates to true. + * + * This is similar to the relation `join` function with one important difference in the result + * schema. Since `joinWith` preserves objects present on either side of the join, the result + * schema is similarly nested into a tuple under the column names `_1` and `_2`. + * + * This type of join can be useful both for preserving type-safety with the original object + * types as well as working with relational data where either side of the join has column names + * in common. + * + * @param other + * Right side of the join. + * @param condition + * Join expression. + * @param joinType + * Type of join to perform. Default `inner`. Must be one of: `inner`, `cross`, `outer`, + * `full`, `fullouter`,`full_outer`, `left`, `leftouter`, `left_outer`, `right`, `rightouter`, + * `right_outer`. + * + * @group typedrel + * @since 3.5.0 + */ + def joinWith[U](other: Dataset[U], condition: Column, joi
[spark] branch master updated (1adf2866915 -> 7bd13530840)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 1adf2866915 [SPARK-44303][SQL] Assign names to the error class _LEGACY_ERROR_TEMP_[2320-2324] add 7bd13530840 [SPARK-44283][CONNECT] Move Origin to SQL/API No new revisions were added by this update. Summary of changes: .../spark/sql/catalyst/trees/SQLQueryContext.scala | 0 .../apache/spark/sql/catalyst/trees/origin.scala | 72 ++ .../apache/spark/sql/catalyst/trees/TreeNode.scala | 54 3 files changed, 72 insertions(+), 54 deletions(-) rename sql/{catalyst => api}/src/main/scala/org/apache/spark/sql/catalyst/trees/SQLQueryContext.scala (100%) create mode 100644 sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/origin.scala - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44284][CONNECT] Create simple conf system for sql/api
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 1fbb94b87c0 [SPARK-44284][CONNECT] Create simple conf system for sql/api 1fbb94b87c0 is described below commit 1fbb94b87c0b10b6f2ded93f0c7eb253d086887e Author: Herman van Hovell AuthorDate: Thu Jul 6 08:22:18 2023 -0400 [SPARK-44284][CONNECT] Create simple conf system for sql/api ### What changes were proposed in this pull request? This PR introduces a configuration system for classes that are in sql/api. ### Why are the changes needed? We are moving a number of components into sql/api that rely on confs being set when used with sql/core. The conf system added here gives us that flexibility to do so. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #41838 from hvanhovell/SPARK-44284. Lead-authored-by: Herman van Hovell Co-authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SqlApiConf.scala| 63 ++ .../spark/sql/catalyst/types/DataTypeUtils.scala | 4 +- .../org/apache/spark/sql/internal/SQLConf.scala| 14 +++-- .../org/apache/spark/sql/types/StructType.scala| 4 +- 4 files changed, 77 insertions(+), 8 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/SqlApiConf.scala new file mode 100644 index 000..07943683ab6 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/SqlApiConf.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import java.util.concurrent.atomic.AtomicReference + +import scala.util.Try + +import org.apache.spark.util.SparkClassUtils + +/** + * Configuration for all objects that are placed in the `sql/api` project. The normal way of + * accessing this class is through `SqlApiConf.get`. If this code is being used with sql/core + * then its values are bound to the currently set SQLConf. With Spark Connect, it will default to + * hardcoded values. + */ +private[sql] trait SqlApiConf { + def ansiEnabled: Boolean + def caseSensitiveAnalysis: Boolean + def maxToStringFields: Int +} + +private[sql] object SqlApiConf { + /** + * Defines a getter that returns the [[SqlApiConf]] within scope. + */ + private val confGetter = new AtomicReference[() => SqlApiConf](() => DefaultSqlApiConf) + + /** + * Sets the active config getter. + */ + private[sql] def setConfGetter(getter: () => SqlApiConf): Unit = { +confGetter.set(getter) + } + + def get: SqlApiConf = confGetter.get()() + + // Force load SQLConf. This will trigger the installation of a confGetter that points to SQLConf. + Try(SparkClassUtils.classForName("org.apache.spark.sql.internal.SQLConf$")) +} + +/** + * Defaults configurations used when no other [[SqlApiConf]] getter is set. + */ +private[sql] object DefaultSqlApiConf extends SqlApiConf { + override def ansiEnabled: Boolean = false + override def caseSensitiveAnalysis: Boolean = false + override def maxToStringFields: Int = 50 +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/DataTypeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/DataTypeUtils.scala index 0d2d6c0262c..da0607e0920 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/DataTypeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/DataTypeUtils.scala @@ -16,9 +16,9 @@ */ package org.apache.spark.sql.catalyst.types +import org.apache.spark.sql.SqlApiConf import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, Literal} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy import org.apache.spark.sql.internal.SQLCo
[spark] branch master updated: [SPARK-44281][SQL] Move QueryCompilation error that used by DataType to sql/api as DataTypeErrors
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 783880b2ff7 [SPARK-44281][SQL] Move QueryCompilation error that used by DataType to sql/api as DataTypeErrors 783880b2ff7 is described below commit 783880b2ff79a62749dd3277c855295f23b813fd Author: Rui Wang AuthorDate: Wed Jul 5 16:40:30 2023 -0400 [SPARK-44281][SQL] Move QueryCompilation error that used by DataType to sql/api as DataTypeErrors ### What changes were proposed in this pull request? Move QueryCompilation error that used by DataType to sql/api as DataTypeErrors. ### Why are the changes needed? To further simplify DataType interface. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Exiting tests. Closes #41835 from amaliujia/compilation_errors. Authored-by: Rui Wang Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/util/QuotingUtils.scala | 31 .../spark/sql/catalyst/util/StringUtils.scala | 6 +++ .../apache/spark/sql/errors/DataTypeErrors.scala | 56 +- .../spark/sql/errors/DataTypeErrorsBase.scala | 29 +++ .../apache/spark/sql/catalyst/util/package.scala | 4 +- .../spark/sql/errors/QueryCompilationErrors.scala | 54 + .../apache/spark/sql/errors/QueryErrorsBase.scala | 32 + .../org/apache/spark/sql/types/DataType.scala | 4 +- .../spark/sql/types/DayTimeIntervalType.scala | 9 ++-- .../org/apache/spark/sql/types/DecimalType.scala | 6 +-- .../spark/sql/types/YearMonthIntervalType.scala| 9 ++-- .../apache/spark/sql/types/StructTypeSuite.scala | 4 +- .../org/apache/spark/sql/JsonFunctionsSuite.scala | 6 +-- 13 files changed, 158 insertions(+), 92 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/QuotingUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/QuotingUtils.scala new file mode 100644 index 000..7d5b6946244 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/QuotingUtils.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.util + +object QuotingUtils { + private def quoteByDefault(elem: String): String = { +"\"" + elem + "\"" + } + + def toSQLConf(conf: String): String = { +quoteByDefault(conf) + } + + def toSQLSchema(schema: String): String = { +quoteByDefault(schema) + } +} diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index 384453e3b53..c12a1f50daa 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -101,4 +101,10 @@ object SparkStringUtils extends Logging { def truncatedString[T](seq: Seq[T], sep: String, maxFields: Int): String = { truncatedString(seq, "", sep, "", maxFields) } + + def quoteIdentifier(name: String): String = { +// Escapes back-ticks within the identifier name with double-back-ticks, and then quote the +// identifier with back-ticks. +"`" + name.replace("`", "``") + "`" + } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala index 02e8b12c707..f39b5b13456 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/DataTypeErrors.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.errors import org.apache.spark.{SparkArithmeticException, SparkException, SparkRuntimeException, SparkUnsupportedOperationException} +import org.apache.spark.s
[spark] branch master updated: [SPARK-44282][CONNECT] Prepare DataType parsing for use in Spark Connect Scala Client
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d1af925f6e1 [SPARK-44282][CONNECT] Prepare DataType parsing for use in Spark Connect Scala Client d1af925f6e1 is described below commit d1af925f6e12ce5ff62c13ffa9ed32e92d548863 Author: Herman van Hovell AuthorDate: Wed Jul 5 16:22:49 2023 -0400 [SPARK-44282][CONNECT] Prepare DataType parsing for use in Spark Connect Scala Client ### What changes were proposed in this pull request? This PR prepares moving DataType parsing to sql/api. Basically it puts all DataType parsing functionality in a super class of regular parsing. We cannot move the parser just yet because that need to happen at the same time as DataType. ### Why are the changes needed? We want the Spark Connect Scala Client to use a restricted class path. DataType will be one of the shared classes, to move DataType we need to move DataType parsing. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Modified existing tests. Closes #41836 from hvanhovell/SPARK-44282. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../sql/catalyst/parser/AbstractSqlParser.scala| 79 .../spark/sql/catalyst/parser/AstBuilder.scala | 168 + .../sql/catalyst/parser/CatalystSqlParser.scala| 27 +++ .../sql/catalyst/parser/DataTypeAstBuilder.scala | 208 + .../catalyst/parser/DataTypeParserInterface.scala | 37 .../sql/catalyst/parser/ParserInterface.scala | 16 +- .../parser/{ParseDriver.scala => parsers.scala}| 71 +-- .../org/apache/spark/sql/types/DataType.scala | 6 +- .../org/apache/spark/sql/types/StructType.scala| 12 +- .../sql/catalyst/parser/DataTypeParserSuite.scala | 2 +- 10 files changed, 369 insertions(+), 257 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala new file mode 100644 index 000..2d6fabaaef6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.parser + +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.parser.ParserUtils.withOrigin +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.Origin +import org.apache.spark.sql.errors.QueryParsingErrors + +/** + * Base class for all ANTLR4 [[ParserInterface]] implementations. + */ +abstract class AbstractSqlParser extends AbstractParser with ParserInterface { + override def astBuilder: AstBuilder + + /** Creates Expression for a given SQL string. */ + override def parseExpression(sqlText: String): Expression = parse(sqlText) { parser => +val ctx = parser.singleExpression() +withOrigin(ctx, Some(sqlText)) { + astBuilder.visitSingleExpression(ctx) +} + } + + /** Creates TableIdentifier for a given SQL string. */ + override def parseTableIdentifier(sqlText: String): TableIdentifier = parse(sqlText) { parser => +astBuilder.visitSingleTableIdentifier(parser.singleTableIdentifier()) + } + + /** Creates FunctionIdentifier for a given SQL string. */ + override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = { +parse(sqlText) { parser => + astBuilder.visitSingleFunctionIdentifier(parser.singleFunctionIdentifier()) +} + } + + /** Creates a multi-part identifier for a given SQL string */ + override def parseMultipartIdentifier(sqlText: String): Seq[String] = { +parse(sqlText) { parser => + astBuilder.visitSingleMultipartIdentifier(parser.singleMultipartIdentif
[spark] branch master updated (9f87e19a5f1 -> d9c2d38f14a)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 9f87e19a5f1 [SPARK-42554][CONNECT] Implement GRPC exceptions interception for conversion add d9c2d38f14a [SPARK-44291][SPARK-43416][CONNECT] Fix incorrect schema for range query No new revisions were added by this update. Summary of changes: .../src/main/scala/org/apache/spark/sql/Dataset.scala | 18 +++--- .../org/apache/spark/sql/ClientE2ETestSuite.scala | 8 .../spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala | 6 ++ 3 files changed, 13 insertions(+), 19 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43757][CONNECT] Change client compatibility from allow list to deny list
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 1fcd537a37b [SPARK-43757][CONNECT] Change client compatibility from allow list to deny list 1fcd537a37b is described below commit 1fcd537a37b2457092e20f8034f23917a8ae2ffa Author: Zhen Li AuthorDate: Wed Jun 28 10:37:38 2023 -0400 [SPARK-43757][CONNECT] Change client compatibility from allow list to deny list ### What changes were proposed in this pull request? Expand the client compatibility check to include all sql APIs. ### Why are the changes needed? Enhance the API compatibility coverage ### Does this PR introduce _any_ user-facing change? No, except it fixes a few wrong types and hides a few helper methods internally. ### How was this patch tested? Existing tests. Closes #41284 from zhenlineo/compatibility-check-allowlist. Authored-by: Zhen Li Signed-off-by: Herman van Hovell --- .../apache/spark/sql/KeyValueGroupedDataset.scala | 6 +- .../scala/org/apache/spark/sql/SparkSession.scala | 2 +- .../sql/streaming/StreamingQueryException.scala| 3 +- .../sql/streaming/StreamingQueryManager.scala | 3 +- .../CheckConnectJvmClientCompatibility.scala | 327 ++--- 5 files changed, 225 insertions(+), 116 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 20c130b83cb..e67ef1c0fa7 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout, OutputMode * * @since 3.5.0 */ -abstract class KeyValueGroupedDataset[K, V] private[sql] () extends Serializable { +class KeyValueGroupedDataset[K, V] private[sql] () extends Serializable { /** * Returns a new [[KeyValueGroupedDataset]] where the type of the key has been mapped to the @@ -462,7 +462,7 @@ abstract class KeyValueGroupedDataset[K, V] private[sql] () extends Serializable UdfUtils.coGroupFunctionToScalaFunc(f))(encoder) } - protected def flatMapGroupsWithStateHelper[S: Encoder, U: Encoder]( + protected[sql] def flatMapGroupsWithStateHelper[S: Encoder, U: Encoder]( outputMode: Option[OutputMode], timeoutConf: GroupStateTimeout, initialState: Option[KeyValueGroupedDataset[K, S]], @@ -923,7 +923,7 @@ private class KeyValueGroupedDatasetImpl[K, V, IK, IV]( agg(aggregator) } - override protected def flatMapGroupsWithStateHelper[S: Encoder, U: Encoder]( + override protected[sql] def flatMapGroupsWithStateHelper[S: Encoder, U: Encoder]( outputMode: Option[OutputMode], timeoutConf: GroupStateTimeout, initialState: Option[KeyValueGroupedDataset[K, S]], diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 45e7dca38d7..54e9102c55c 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -429,7 +429,7 @@ class SparkSession private[sql] ( * * @since 3.4.0 */ - object implicits extends SQLImplicits(this) + object implicits extends SQLImplicits(this) with Serializable // scalastyle:on def newSession(): SparkSession = { diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala index d5e9982dfbf..512c94f5c70 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala @@ -36,7 +36,8 @@ class StreamingQueryException private[sql] ( message: String, errorClass: String, stackTrace: String) -extends SparkThrowable { +extends Exception(message) +with SparkThrowable { override def getErrorClass: String = errorClass diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 775921ff579..13bbf470639 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming
[spark] branch master updated: [SPARK-44039][CONNECT][TESTS] Improve for PlanGenerationTestSuite & ProtoToParsedPlanTestSuite
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c8c51d04741 [SPARK-44039][CONNECT][TESTS] Improve for PlanGenerationTestSuite & ProtoToParsedPlanTestSuite c8c51d04741 is described below commit c8c51d047411959ad6c648246a5bd6ea4ae13ce8 Author: panbingkun AuthorDate: Tue Jun 27 21:37:33 2023 -0400 [SPARK-44039][CONNECT][TESTS] Improve for PlanGenerationTestSuite & ProtoToParsedPlanTestSuite ### What changes were proposed in this pull request? The pr aims to improve for PlanGenerationTestSuite & ProtoToParsedPlanTestSuite, include: - When generating `GOLDEN` files, we should first delete the corresponding directories and generate new ones to avoid submitting some redundant files during the review process. eg: When we write a test named `make_timestamp_ltz` for the overloaded method, and during the review process, the reviewer wishes to add more tests for the method. The name of this method has changed during the next submission process, such as `make_timestamp_ltz without timezone`.At this point, if the `queries/function_make_timestamp_ltz.json`, `queries/function_make_timestamp_ltz.proto.bin` and `explain-results/function_make_timestamp_ltz.explain` files of `function_make_timestamp_ltz` [...] - Clear and update some redundant files submitted incorrectly ### Why are the changes needed? Make code clear. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. Closes #41572 from panbingkun/SPARK-44039. Authored-by: panbingkun Signed-off-by: Herman van Hovell --- .../apache/spark/sql/PlanGenerationTestSuite.scala | 28 ++ .../explain-results/function_percentile.explain| 2 - .../function_regexp_extract_all.explain| 2 - .../explain-results/function_regexp_instr.explain | 2 - .../query-tests/explain-results/read_path.explain | 1 - .../query-tests/queries/function_lit_array.json| 58 ++--- .../query-tests/queries/function_percentile.json | 29 --- .../queries/function_percentile.proto.bin | Bin 192 -> 0 bytes .../queries/function_regexp_extract_all.json | 33 .../queries/function_regexp_extract_all.proto.bin | Bin 212 -> 0 bytes .../query-tests/queries/function_regexp_instr.json | 33 .../queries/function_regexp_instr.proto.bin| Bin 203 -> 0 bytes .../resources/query-tests/queries/read_path.json | 11 .../query-tests/queries/read_path.proto.bin| 3 -- .../queries/streaming_table_API_with_options.json | 3 +- .../sql/connect/ProtoToParsedPlanTestSuite.scala | 23 16 files changed, 82 insertions(+), 146 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala index e8d04f37d7f..ecb7092b8d9 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala @@ -44,6 +44,7 @@ import org.apache.spark.sql.functions.lit import org.apache.spark.sql.protobuf.{functions => pbFn} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.util.Utils // scalastyle:off /** @@ -61,6 +62,14 @@ import org.apache.spark.unsafe.types.CalendarInterval * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "connect-client-jvm/testOnly org.apache.spark.sql.PlanGenerationTestSuite" * }}} * + * If you need to clean the orphaned golden files, you need to set the + * SPARK_CLEAN_ORPHANED_GOLDEN_FILES=1 environment variable before running this test, e.g.: + * {{{ + * SPARK_CLEAN_ORPHANED_GOLDEN_FILES=1 build/sbt "connect-client-jvm/testOnly org.apache.spark.sql.PlanGenerationTestSuite" + * }}} + * Note: not all orphaned golden files should be cleaned, some are reserved for testing backups + * compatibility. + * * Note that the plan protos are used as the input for the `ProtoToParsedPlanTestSuite` in the * `connector/connect/server` module */ @@ -74,6 +83,9 @@ class PlanGenerationTestSuite // Borrowed from SparkFunSuite private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" + private val cleanOrphanedGoldenFiles: Boolean = +System.getenv("SPARK_CLEAN_ORPHANED_GOLDEN_FILES") == "1" + protected val queryFilePath: Path = commonResourcePath.resolve("query-tests/queries") // A relative path to /con
[spark] branch master updated: [SPARK-44161][CONNECT] Handle Row input for UDFs
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 05fc3497f00 [SPARK-44161][CONNECT] Handle Row input for UDFs 05fc3497f00 is described below commit 05fc3497f00a0aad9240f14637ea21d271b2bbe4 Author: Zhen Li AuthorDate: Tue Jun 27 21:05:33 2023 -0400 [SPARK-44161][CONNECT] Handle Row input for UDFs ### What changes were proposed in this pull request? If the client passes Rows as inputs to UDFs, the Spark connect planner will fail to create the RowEncoder for the Row input. The Row encoder sent by the client contains no field or schema information. The real input schema should be obtained from the plan's output. This PR ensures if the server planner failed to create the encoder for the UDF input using reflection, then it will fall back to use RowEncoders created from the plan.output schema. This PR fixed [SPARK-43761](https://issues.apache.org/jira/browse/SPARK-43761) using the same logic. This PR resolved [SPARK-43796](https://issues.apache.org/jira/browse/SPARK-43796). The error is just caused by the case class defined in the test. ### Why are the changes needed? Fix the bug where the Row cannot be used as UDF inputs. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? E2E tests. Closes #41704 from zhenlineo/rowEncoder. Authored-by: Zhen Li Signed-off-by: Herman van Hovell --- .../sql/expressions/UserDefinedFunction.scala | 5 +- .../spark/sql/streaming/DataStreamWriter.scala | 11 +-- .../sql/KeyValueGroupedDatasetE2ETestSuite.scala | 45 + .../sql/UserDefinedFunctionE2ETestSuite.scala | 36 +- .../spark/sql/streaming/StreamingQuerySuite.scala | 55 --- .../sql/connect/planner/SparkConnectPlanner.scala | 78 ++ .../spark/sql/catalyst/ScalaReflection.scala | 38 --- .../spark/sql/catalyst/ScalaReflectionSuite.scala | 21 +- .../spark/sql/streaming/DataStreamWriter.scala | 6 +- 9 files changed, 210 insertions(+), 85 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index bfcd4572e03..14dfc0c6a86 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -142,7 +142,10 @@ object ScalarUserDefinedFunction { ScalarUserDefinedFunction( function = function, - inputEncoders = parameterTypes.map(tag => ScalaReflection.encoderFor(tag)), + // Input can be a row because the input data schema can be found from the plan. + inputEncoders = +parameterTypes.map(tag => ScalaReflection.encoderForWithRowEncoderSupport(tag)), + // Output cannot be a row as there is no good way to get the return data type. outputEncoder = ScalaReflection.encoderFor(returnType)) } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 263e1e372c8..ed3d2bb8558 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -30,7 +30,6 @@ import org.apache.spark.connect.proto.Command import org.apache.spark.connect.proto.WriteStreamOperationStart import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, ForeachWriter} -import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.UnboundRowEncoder import org.apache.spark.sql.connect.common.ForeachWriterPacket import org.apache.spark.sql.execution.streaming.AvailableNowTrigger import org.apache.spark.sql.execution.streaming.ContinuousTrigger @@ -215,15 +214,7 @@ final class DataStreamWriter[T] private[sql] (ds: Dataset[T]) extends Logging { * @since 3.5.0 */ def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = { -// TODO [SPARK-43761] Update this once resolved UnboundRowEncoder serialization issue. -// ds.encoder equal to UnboundRowEncoder means type parameter T is Row, -// which is not able to be serialized. Server will detect this and use default encoder. -val rowEncoder = if (ds.encoder != UnboundRowEncoder) { - ds.encoder -} else { - null -} -val serialized = Utils.serialize(ForeachWriterPacket(writer, rowEncoder
[spark] branch master updated: [SPARK-43136][CONNECT][FOLLOWUP] Adding tests for KeyAs
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a9199c3d815 [SPARK-43136][CONNECT][FOLLOWUP] Adding tests for KeyAs a9199c3d815 is described below commit a9199c3d815b0f63ba2ddbad2532e6d12180aafe Author: Zhen Li AuthorDate: Mon Jun 26 22:54:23 2023 -0400 [SPARK-43136][CONNECT][FOLLOWUP] Adding tests for KeyAs ### What changes were proposed in this pull request? The current impl of KeyAs for the Scala client is a purely client side encoder operation. Thus we could end up with duplicates in keys. Added the tests both for the Scala Client and for Spark dataset API. It showed the behavior is the same for server and client at this moment. ### Why are the changes needed? More tests to verify the client and server behavior. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tests Closes #40980 from zhenlineo/keys. Authored-by: Zhen Li Signed-off-by: Herman van Hovell --- .../sql/KeyValueGroupedDatasetE2ETestSuite.scala | 29 ++ .../scala/org/apache/spark/sql/DatasetSuite.scala | 24 ++ 2 files changed, 53 insertions(+) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala index 404239f7e14..173867060bd 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala @@ -79,6 +79,32 @@ class KeyValueGroupedDatasetE2ETestSuite extends QueryTest with SQLHelper { assert(values == Arrays.asList[Double](0, 1)) } + test("groupByKey, keyAs - duplicates") { +val session: SparkSession = spark +import session.implicits._ +val result = spark + .range(10) + .as[Long] + .groupByKey(id => K2(id % 2, id % 4)) + .keyAs[K1] + .flatMapGroups((_, it) => Seq(it.toSeq.size)) + .collect() +assert(result.sorted === Seq(2, 2, 3, 3)) + } + + test("groupByKey, keyAs, keys - duplicates") { +val session: SparkSession = spark +import session.implicits._ +val result = spark + .range(10) + .as[Long] + .groupByKey(id => K2(id % 2, id % 4)) + .keyAs[K1] + .keys + .collect() +assert(result.sortBy(_.a) === Seq(K1(0), K1(0), K1(1), K1(1))) + } + test("keyAs - flatGroupMap") { val values = spark .range(10) @@ -555,3 +581,6 @@ class KeyValueGroupedDatasetE2ETestSuite extends QueryTest with SQLHelper { checkDataset(values, ClickState("a", 5), ClickState("b", 3), ClickState("c", 1)) } } + +case class K1(a: Long) +case class K2(a: Long, b: Long) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 167aea79209..0766dd2e772 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -668,6 +668,27 @@ class DatasetSuite extends QueryTest ) } + test("groupByKey, keyAs - duplicates") { +val ds = spark + .range(10) + .as[Long] + .groupByKey(id => K2(id % 2, id % 4)) + .keyAs[K1] + .flatMapGroups((_, it) => Seq(it.toSeq.size)) +checkDatasetUnorderly(ds, 3, 2, 3, 2) + } + + test("groupByKey, keyAs, keys - duplicates") { +val result = spark + .range(10) + .as[Long] + .groupByKey(id => K2(id % 2, id % 4)) + .keyAs[K1] + .keys + .collect() +assert(result.sortBy(_.a) === Seq(K1(0), K1(0), K1(1), K1(1))) + } + test("groupBy function, mapValues, flatMap") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() val keyValue = ds.groupByKey(_._1).mapValues(_._2) @@ -2626,3 +2647,6 @@ case class SpecialCharClass(`field.1`: String, `field 2`: String) /** Used to test Java Enums from Scala code */ case class SaveModeCase(mode: SaveMode) case class SaveModeArrayCase(modes: Array[SaveMode]) + +case class K1(a: Long) +case class K2(a: Long, b: Long) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44030][SQL] Implement DataTypeExpression to offer Unapply for expression
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 686e37e640d [SPARK-44030][SQL] Implement DataTypeExpression to offer Unapply for expression 686e37e640d is described below commit 686e37e640d078f9727e5457e47ce58033ce8684 Author: Rui Wang AuthorDate: Mon Jun 26 22:47:01 2023 -0400 [SPARK-44030][SQL] Implement DataTypeExpression to offer Unapply for expression ### What changes were proposed in this pull request? Implement DataTypeExpression to offer `Unapply` for expression. By doing so we can drop `Unapply` from DataType. ### Why are the changes needed? Simplify DataType interface. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? existing tests Closes #41559 from amaliujia/move_datatypes_1. Authored-by: Rui Wang Signed-off-by: Herman van Hovell --- .../sql/catalyst/analysis/AnsiTypeCoercion.scala | 29 +- .../sql/catalyst/analysis/DecimalPrecision.scala | 20 +++ .../spark/sql/catalyst/analysis/TypeCoercion.scala | 46 +++ .../apache/spark/sql/types/AbstractDataType.scala | 40 + .../org/apache/spark/sql/types/DataType.scala | 10 .../spark/sql/types/DataTypeExpression.scala | 67 ++ .../apache/spark/sql/hive/client/HiveShim.scala| 4 +- 7 files changed, 119 insertions(+), 97 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala index 56dbb2a8590..d3f20f87493 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala @@ -244,28 +244,29 @@ object AnsiTypeCoercion extends TypeCoercionBase { val promoteType = findWiderTypeForString(left.dataType, right.dataType).get b.withNewChildren(Seq(castExpr(left, promoteType), castExpr(right, promoteType))) - case Abs(e @ StringType(), failOnError) => Abs(Cast(e, DoubleType), failOnError) - case m @ UnaryMinus(e @ StringType(), _) => m.withNewChildren(Seq(Cast(e, DoubleType))) - case UnaryPositive(e @ StringType()) => UnaryPositive(Cast(e, DoubleType)) + case Abs(e @ StringTypeExpression(), failOnError) => Abs(Cast(e, DoubleType), failOnError) + case m @ UnaryMinus(e @ StringTypeExpression(), _) => +m.withNewChildren(Seq(Cast(e, DoubleType))) + case UnaryPositive(e @ StringTypeExpression()) => UnaryPositive(Cast(e, DoubleType)) - case d @ DateAdd(left @ StringType(), _) => + case d @ DateAdd(left @ StringTypeExpression(), _) => d.copy(startDate = Cast(d.startDate, DateType)) - case d @ DateAdd(_, right @ StringType()) => + case d @ DateAdd(_, right @ StringTypeExpression()) => d.copy(days = Cast(right, IntegerType)) - case d @ DateSub(left @ StringType(), _) => + case d @ DateSub(left @ StringTypeExpression(), _) => d.copy(startDate = Cast(d.startDate, DateType)) - case d @ DateSub(_, right @ StringType()) => + case d @ DateSub(_, right @ StringTypeExpression()) => d.copy(days = Cast(right, IntegerType)) - case s @ SubtractDates(left @ StringType(), _, _) => + case s @ SubtractDates(left @ StringTypeExpression(), _, _) => s.copy(left = Cast(s.left, DateType)) - case s @ SubtractDates(_, right @ StringType(), _) => + case s @ SubtractDates(_, right @ StringTypeExpression(), _) => s.copy(right = Cast(s.right, DateType)) - case t @ TimeAdd(left @ StringType(), _, _) => + case t @ TimeAdd(left @ StringTypeExpression(), _, _) => t.copy(start = Cast(t.start, TimestampType)) - case t @ SubtractTimestamps(left @ StringType(), _, _, _) => + case t @ SubtractTimestamps(left @ StringTypeExpression(), _, _, _) => t.copy(left = Cast(t.left, t.right.dataType)) - case t @ SubtractTimestamps(_, right @ StringType(), _, _) => + case t @ SubtractTimestamps(_, right @ StringTypeExpression(), _, _) => t.copy(right = Cast(right, t.left.dataType)) } } @@ -296,9 +297,9 @@ object AnsiTypeCoercion extends TypeCoercionBase { case d @ DateAdd(AnyTimestampType(), _) => d.copy(startDate = Cast(d.startDate, DateType)) case d @ DateSub(AnyTimestampType(), _) => d.copy(startDate = Cast(d.startDate, DateType)) - case s @ SubtractTimestamps(DateType(), AnyTimestampType(), _, _) => + case s @ SubtractTimestamps(DateTypeExpression(), A
[spark] branch master updated: [SPARK-44146][CONNECT] Isolate Spark Connect Session jars and classfiles
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new b02ea4cd370 [SPARK-44146][CONNECT] Isolate Spark Connect Session jars and classfiles b02ea4cd370 is described below commit b02ea4cd370ce6a066561dfde9d517ea70805a2b Author: vicennial AuthorDate: Mon Jun 26 21:42:19 2023 -0400 [SPARK-44146][CONNECT] Isolate Spark Connect Session jars and classfiles ### What changes were proposed in this pull request? This PR follows up on https://github.com/apache/spark/pull/41625 to utilize the classloader/resource isolation in Spark to support multi-user Spark Connect sessions which are isolated from each other (currently, classfiles and jars) and thus, enables multi-user REPLs and UDFs. - Instead of a single instance of `SparkArtifactManager` handling all the artifact movement, each instance is now responsible for a single `sessionHolder` (i.e a Spark Connect session) which it requires in it's constructor. - Previously, all artifacts were stored under a common directory `sparkConnectArtifactDirectory` which was initialised in `SparkContext`. Moving forward, all artifacts are instead separated based on the underlying `SparkSession` (using it's `sessionUUID`) they belong to in the format of `ROOT_ARTIFACT_DIR//jars/...`. - The `SparkConnectArtifactManager` also builds a `JobArtifactSet` [here](https://github.com/apache/spark/pull/41701/files#diff-f833145e80f2b42f54f446a0f173e60e3f5ad657a6ad1f2135bc5c20bcddc90cR157-R168) which is eventually propagated to the executors where the classloader isolation mechanism uses the `uuid` parameter. - Currently, classfile and jars are isolated but files and archives aren't. ### Why are the changes needed? To enable support for multi-user sessions coexisting on a singular Spark cluster. For example, multi-user Scala REPLs/UDFs will be supported with this PR. ### Does this PR introduce _any_ user-facing change? Yes, multiple Spark Connect REPLs may use a single Spark cluster at once and execute their own UDFs without intefering with each other. ### How was this patch tested? New unit tests in `ArtifactManagerSuite` + existing tests. Closes #41701 from vicennial/SPARK-44146. Authored-by: vicennial Signed-off-by: Herman van Hovell --- .../artifact/SparkConnectArtifactManager.scala | 205 + .../sql/connect/planner/SparkConnectPlanner.scala | 19 +- .../spark/sql/connect/service/SessionHolder.scala | 103 +++ .../service/SparkConnectAddArtifactsHandler.scala | 7 +- .../service/SparkConnectAnalyzeHandler.scala | 21 ++- .../sql/connect/service/SparkConnectService.scala | 4 +- .../service/SparkConnectStreamHandler.scala| 15 +- .../connect/artifact/ArtifactManagerSuite.scala| 167 ++--- .../scala/org/apache/spark/JobArtifactSet.scala| 19 +- .../main/scala/org/apache/spark/SparkContext.scala | 23 +-- .../main/scala/org/apache/spark/rpc/RpcEnv.scala | 11 ++ .../spark/rpc/netty/NettyStreamManager.scala | 5 + 12 files changed, 441 insertions(+), 158 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala index 05c0a597722..0a91c6b9550 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala @@ -26,9 +26,12 @@ import javax.ws.rs.core.UriBuilder import scala.collection.JavaConverters._ import scala.reflect.ClassTag +import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.{LocalFileSystem, Path => FSPath} -import org.apache.spark.{SparkContext, SparkEnv} +import org.apache.spark.{JobArtifactSet, SparkContext, SparkEnv} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connect.artifact.util.ArtifactUtils import org.apache.spark.sql.connect.config.Connect.CONNECT_COPY_FROM_LOCAL_TO_FS_ALLOW_DEST_LOCAL import org.apache.spark.sql.connect.service.SessionHolder @@ -39,45 +42,29 @@ import org.apache.spark.util.Utils * The Artifact Manager for the [[SparkConnectService]]. * * This class handles the storage of artifacts as well as preparing the artifacts for use. - * Currently, jars and classfile artifacts undergo additional processing: - * - Jars and pyfiles are automatically added to the underlying [[SparkContext]] and are - * accessible by all users of the cluster. - * - Cla
[spark] branch master updated (67abc430140 -> 2994560d204)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 67abc430140 [SPARK-44143][SQL][TESTS] Use checkError() to check Exception in *DDL*Suite add 2994560d204 [SPARK-44164][SQL] Extract toAttribute method from StructField to Util class No new revisions were added by this update. Summary of changes: .../spark/sql/catalyst/expressions/namedExpressions.scala | 2 +- .../org/apache/spark/sql/catalyst/types/DataTypeUtils.scala| 10 -- .../main/scala/org/apache/spark/sql/types/StructField.scala| 4 .../src/main/scala/org/apache/spark/sql/types/StructType.scala | 3 ++- .../spark/sql/execution/datasources/FileSourceStrategy.scala | 5 +++-- 5 files changed, 14 insertions(+), 10 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44078][CONNECT][CORE] Add support for classloader/resource isolation
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9d031ba8c99 [SPARK-44078][CONNECT][CORE] Add support for classloader/resource isolation 9d031ba8c99 is described below commit 9d031ba8c995286e5f8892764e5108aa60f49238 Author: vicennial AuthorDate: Wed Jun 21 20:58:19 2023 -0400 [SPARK-44078][CONNECT][CORE] Add support for classloader/resource isolation ### What changes were proposed in this pull request? This PR adds a `JobArtifactSet` which holds the jars/files/archives relevant to a particular Spark Job. Using this "set", we are able to support specifying visible/available resources for a job based on, for example, the SparkSession that the job belongs to. With resource specification support, we are further able to extend this to support classloader/resource isolation on the executors. The executors would use the `uuid` from the `JobArtifactSet` to either create or obtain from a cache the [IsolatedSessionState](https://github.com/apache/spark/pull/41625/files#diff-d7a989c491f3cb77cca02c701496a9e2a3443f70af73b0d1ab0899239f3a789dR57) which holds the "state" (i.e classloaders, files, jars, archives etc) for that particular `uuid`. Currently, the code will default to copying over resources from the `SparkContext` (the current/default behaviour) to avoid any behaviour changes. A follow-up PR would use this mechanism in Spark Connect to isolate resources among Spark Connect sessions. ### Why are the changes needed? A current limitation of Scala UDFs is that a Spark cluster would only be able to support a single REPL at a time due to the fact that classloaders of different Spark Sessions (and therefore, Spark Connect sessions) aren't isolated from each other. Without isolation, REPL-generated class files and user-added JARs may conflict if there are multiple users of the cluster. Thus, we need a mechanism to support isolated sessions (i.e isolated resources/classloader) so that each REPL user does not conflict with other users on the same cluster. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests + new suite `JobArtifactSetSuite`. Closes #41625 from vicennial/SPARK-44078. Authored-by: vicennial Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/JobArtifactSet.scala| 123 + .../scala/org/apache/spark/executor/Executor.scala | 120 +--- .../org/apache/spark/scheduler/ActiveJob.scala | 3 + .../org/apache/spark/scheduler/DAGScheduler.scala | 37 --- .../apache/spark/scheduler/DAGSchedulerEvent.scala | 2 + .../org/apache/spark/scheduler/ResultTask.scala| 5 +- .../apache/spark/scheduler/ShuffleMapTask.scala| 9 +- .../scala/org/apache/spark/scheduler/Task.scala| 2 + .../apache/spark/scheduler/TaskDescription.scala | 61 ++ .../apache/spark/scheduler/TaskSetManager.scala| 9 +- .../org/apache/spark/JobArtifactSetSuite.scala | 87 +++ .../CoarseGrainedExecutorBackendSuite.scala| 7 +- .../org/apache/spark/executor/ExecutorSuite.scala | 21 ++-- .../CoarseGrainedSchedulerBackendSuite.scala | 12 +- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 9 +- .../org/apache/spark/scheduler/FakeTask.scala | 8 +- .../spark/scheduler/NotSerializableFakeTask.scala | 4 +- .../apache/spark/scheduler/TaskContextSuite.scala | 26 +++-- .../spark/scheduler/TaskDescriptionSuite.scala | 18 +-- .../spark/scheduler/TaskSchedulerImplSuite.scala | 4 +- .../spark/scheduler/TaskSetManagerSuite.scala | 12 +- .../MesosFineGrainedSchedulerBackendSuite.scala| 10 +- 22 files changed, 436 insertions(+), 153 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/JobArtifactSet.scala b/core/src/main/scala/org/apache/spark/JobArtifactSet.scala new file mode 100644 index 000..d87c25c0b7c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/JobArtifactSet.scala @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT
[spark] branch master updated (fb1ee25a89e -> 01d068b06d5)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from fb1ee25a89e [SPARK-43290][SQL] Adds support for aes_encrypt IVs and AAD add 01d068b06d5 [SPARK-43474][SS][CONNECT] Add SessionHolder to SparkConnectPlanner No new revisions were added by this update. Summary of changes: .../spark/sql/connect/planner/SparkConnectPlanner.scala | 5 - .../spark/sql/connect/service/SessionHolder.scala | 8 .../connect/service/SparkConnectAnalyzeHandler.scala| 17 - .../sql/connect/service/SparkConnectStreamHandler.scala | 17 + .../spark/sql/connect/ProtoToParsedPlanTestSuite.scala | 3 ++- .../sql/connect/planner/SparkConnectPlannerSuite.scala | 6 -- .../sql/connect/planner/SparkConnectServiceSuite.scala | 17 ++--- .../plugin/SparkConnectPluginRegistrySuite.scala| 4 +++- 8 files changed, 48 insertions(+), 29 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (a02275ac16c -> dfd40a40946)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from a02275ac16c [SPARK-43380][SQL] Fix conversion of Avro logical timestamp type to Long add dfd40a40946 [SPARK-44016][CONNECT] Prevent processing of artifacts that contain invalid paths (absolute, parent, sibling, nephew) No new revisions were added by this update. Summary of changes: .../artifact/SparkConnectArtifactManager.scala | 8 ++- .../sql/connect/artifact/util/ArtifactUtils.scala | 43 .../service/SparkConnectAddArtifactsHandler.scala | 12 +++- .../connect/service/AddArtifactsHandlerSuite.scala | 81 ++ 4 files changed, 140 insertions(+), 4 deletions(-) create mode 100644 connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/util/ArtifactUtils.scala - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43888][CONNECT][FOLLOW-UP] Spark Connect client should depend on common-utils explicitly
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2d250085f0a [SPARK-43888][CONNECT][FOLLOW-UP] Spark Connect client should depend on common-utils explicitly 2d250085f0a is described below commit 2d250085f0a2d3f0f124e274b029bbed29067a6d Author: Rui Wang AuthorDate: Tue Jun 13 11:19:05 2023 -0400 [SPARK-43888][CONNECT][FOLLOW-UP] Spark Connect client should depend on common-utils explicitly ### What changes were proposed in this pull request? Spark Connect client should depend on common-utils directly. ### Why are the changes needed? Spark Connect client is already depending on `common-utils` transitively. We should make this dependency explicit. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? N/A Closes #41427 from amaliujia/add_common_util_depends. Authored-by: Rui Wang Signed-off-by: Herman van Hovell --- connector/connect/client/jvm/pom.xml | 6 ++ 1 file changed, 6 insertions(+) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index cbb283ed993..8a51bf65d6a 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -59,6 +59,11 @@ + + org.apache.spark + spark-common-utils_${scala.binary.version} + ${project.version} + com.google.protobuf protobuf-java @@ -160,6 +165,7 @@ org.codehaus.mojo:* org.checkerframework:* org.apache.spark:spark-connect-common_${scala.binary.version} + org.apache.spark:spark-common-utils_${scala.binary.version} - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43920][SQL][CONNECT] Create sql/api module
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 15202e53195 [SPARK-43920][SQL][CONNECT] Create sql/api module 15202e53195 is described below commit 15202e531957849e9eaefcaa6fa1c522a8967d80 Author: Rui Wang AuthorDate: Wed Jun 7 20:53:02 2023 -0400 [SPARK-43920][SQL][CONNECT] Create sql/api module ### What changes were proposed in this pull request? We need a sql/api module to host public API like DataType, Row, etc. This module can be shared between Catalyst and Spark Connect client so that client do not need to depend on Catalyst anymore. ### Why are the changes needed? Towards Spark Connect client do not need to depend on Catalyst anymore. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? N/A Closes #41426 from amaliujia/add_sql_api. Authored-by: Rui Wang Signed-off-by: Herman van Hovell --- pom.xml | 1 + project/SparkBuild.scala | 6 +++--- sql/api/pom.xml | 45 + 3 files changed, 49 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 175df1722e6..3c87da45bea 100644 --- a/pom.xml +++ b/pom.xml @@ -89,6 +89,7 @@ mllib-local tools streaming +sql/api sql/catalyst sql/core sql/hive diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c4c19c65bf1..023ce4ba81c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -58,10 +58,10 @@ object BuildCommons { val allProjects@Seq( core, graphx, mllib, mllibLocal, repl, networkCommon, networkShuffle, launcher, unsafe, tags, sketch, kvstore, -commonUtils, _* +commonUtils, sqlApi, _* ) = Seq( "core", "graphx", "mllib", "mllib-local", "repl", "network-common", "network-shuffle", "launcher", "unsafe", -"tags", "sketch", "kvstore", "common-utils" +"tags", "sketch", "kvstore", "common-utils", "sql-api" ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects ++ Seq(connectCommon, connect, connectClient) val optionallyEnabledProjects@Seq(kubernetes, mesos, yarn, @@ -408,7 +408,7 @@ object SparkBuild extends PomBuild { Seq( spark, hive, hiveThriftServer, repl, networkCommon, networkShuffle, networkYarn, unsafe, tags, tokenProviderKafka010, sqlKafka010, connectCommon, connect, connectClient, protobuf, - commonUtils + commonUtils, sqlApi ).contains(x) } diff --git a/sql/api/pom.xml b/sql/api/pom.xml new file mode 100644 index 000..9d100b1130e --- /dev/null +++ b/sql/api/pom.xml @@ -0,0 +1,45 @@ + + + +http://maven.apache.org/POM/4.0.0"; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd";> +4.0.0 + +org.apache.spark +spark-parent_2.12 +3.5.0-SNAPSHOT +../../pom.xml + + +spark-sql-api_2.12 +jar +Spark Project SQL API +https://spark.apache.org/ + +sql-api + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + \ No newline at end of file - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (ae8aaec9cde -> 8c6a54d70a7)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from ae8aaec9cde [SPARK-43977][CONNECT] Fix unexpected check result of `dev/connect-jvm-client-mima-check` add 8c6a54d70a7 [SPARK-43919][SQL] Extract JSON functionality out of Row No new revisions were added by this update. Summary of changes: .../org/apache/spark/sql/streaming/progress.scala | 5 +- docs/sql-migration-guide.md| 1 + project/MimaExcludes.scala | 6 + .../src/main/scala/org/apache/spark/sql/Row.scala | 106 + .../org/apache/spark/sql/util/ToJsonUtil.scala | 129 + .../scala/org/apache/spark/sql/RowJsonSuite.scala | 7 +- .../test/scala/org/apache/spark/sql/RowTest.scala | 3 +- .../org/apache/spark/sql/streaming/progress.scala | 3 +- 8 files changed, 149 insertions(+), 111 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/util/ToJsonUtil.scala - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-42626][CONNECT] Add Destructive Iterator for SparkResult
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 62338ed6cd9 [SPARK-42626][CONNECT] Add Destructive Iterator for SparkResult 62338ed6cd9 is described below commit 62338ed6cd9fba8bb92ec11cea643077e4b69db4 Author: Tengfei Huang AuthorDate: Mon Jun 5 21:30:02 2023 -0400 [SPARK-42626][CONNECT] Add Destructive Iterator for SparkResult ### What changes were proposed in this pull request? Add a destructive iterator to SparkResult and change `Dataset.toLocalIterator` to use the desctructive iterator. With the desctructive iterator, we will: 1. Close the `ColumarBatch` once its data got consumed; 2. Remove the `ColumarBatch` from `SparkResult.batches`; ### Why are the changes needed? Instead of keeping everything in memory for the life time of SparkResult object, clean it up as soon as we know we are done with it. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT added. Closes #40610 from ivoson/SPARK-42626. Authored-by: Tengfei Huang Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 3 +- .../spark/sql/connect/client/SparkResult.scala | 43 -- .../org/apache/spark/sql/ClientE2ETestSuite.scala | 52 +- 3 files changed, 81 insertions(+), 17 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 7a680bde7d3..eba425ce127 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2768,8 +2768,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def toLocalIterator(): java.util.Iterator[T] = { -// TODO make this a destructive iterator. -collectResult().iterator +collectResult().destructiveIterator } /** diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index 49db44bd855..86a7cf846f2 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -46,7 +46,8 @@ private[sql] class SparkResult[T]( private[this] var numRecords: Int = 0 private[this] var structType: StructType = _ private[this] var boundEncoder: ExpressionEncoder[T] = _ - private[this] val batches = mutable.Buffer.empty[ColumnarBatch] + private[this] var nextBatchIndex: Int = 0 + private val idxToBatches = mutable.Map.empty[Int, ColumnarBatch] private def createEncoder(schema: StructType): ExpressionEncoder[T] = { val agnosticEncoder = if (encoder == UnboundRowEncoder) { @@ -70,12 +71,12 @@ private[sql] class SparkResult[T]( val reader = new ArrowStreamReader(ipcStreamBytes.newInput(), allocator) try { val root = reader.getVectorSchemaRoot - if (batches.isEmpty) { -if (structType == null) { - // If the schema is not available yet, fallback to the schema from Arrow. - structType = ArrowUtils.fromArrowSchema(root.getSchema) -} -// TODO: create encoders that directly operate on arrow vectors. + if (structType == null) { +// If the schema is not available yet, fallback to the schema from Arrow. +structType = ArrowUtils.fromArrowSchema(root.getSchema) + } + // TODO: create encoders that directly operate on arrow vectors. + if (boundEncoder == null) { boundEncoder = createEncoder(structType).resolveAndBind(structType.toAttributes) } while (reader.loadNextBatch()) { @@ -85,7 +86,8 @@ private[sql] class SparkResult[T]( val vectors = root.getFieldVectors.asScala .map(v => new ArrowColumnVector(transferToNewVector(v))) .toArray[ColumnVector] - batches += new ColumnarBatch(vectors, rowCount) + idxToBatches.put(nextBatchIndex, new ColumnarBatch(vectors, rowCount)) + nextBatchIndex += 1 numRecords += rowCount if (stopOnFirstNonEmptyResponse) { return true @@ -142,24 +144,39 @@ private[sql] class SparkResult[T]( /** * Returns an iterator over the contents of the result. */ - def iterator: java.util.Iterator[T] with AutoCloseable = { + def iterator: java.util.Iterator[T] w
[spark] branch branch-3.4 updated: [SPARK-43894][PYTHON] Fix bug in df.cache()
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch branch-3.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.4 by this push: new 0e1401dc71b [SPARK-43894][PYTHON] Fix bug in df.cache() 0e1401dc71b is described below commit 0e1401dc71b5aee540a54fc6a36a1857b13390b4 Author: Martin Grund AuthorDate: Wed May 31 11:55:19 2023 -0400 [SPARK-43894][PYTHON] Fix bug in df.cache() ### What changes were proposed in this pull request? Previously calling `df.cache()` would result in an invalid plan input exception because we did not invoke `persist()` with the right arguments. This patch simplifies the logic and makes it compatible to the behavior in Spark itself. ### Why are the changes needed? Bug ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added UT Closes #41399 from grundprinzip/df_cache. Authored-by: Martin Grund Signed-off-by: Herman van Hovell (cherry picked from commit d3f76c6ca07a7a11fd228dde770186c0fbc3f03f) Signed-off-by: Herman van Hovell --- python/pyspark/sql/connect/dataframe.py| 4 +--- python/pyspark/sql/tests/connect/test_connect_basic.py | 6 ++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index ca2e1b7a0dc..03049109061 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -1544,9 +1544,7 @@ class DataFrame: def cache(self) -> "DataFrame": if self._plan is None: raise Exception("Cannot cache on empty plan.") -relation = self._plan.plan(self._session.client) -self._session.client._analyze(method="persist", relation=relation) -return self +return self.persist() cache.__doc__ = PySparkDataFrame.cache.__doc__ diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index 008b95d6f14..b051b9233c8 100644 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -3032,6 +3032,12 @@ class SparkConnectBasicTests(SparkConnectSQLTestCase): message_parameters={"attr_name": "_jreader"}, ) +def test_df_caache(self): +df = self.connect.range(10) +df.cache() +self.assert_eq(10, df.count()) +self.assertTrue(df.is_cached) + class SparkConnectSessionTests(SparkConnectSQLTestCase): def _check_no_active_session_error(self, e: PySparkException): - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (fead25ac4d6 -> d3f76c6ca07)
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from fead25ac4d6 [SPARK-43775][SQL] DataSource V2: Allow representing updates as deletes and inserts add d3f76c6ca07 [SPARK-43894][PYTHON] Fix bug in df.cache() No new revisions were added by this update. Summary of changes: python/pyspark/sql/connect/dataframe.py| 4 +--- python/pyspark/sql/tests/connect/test_connect_basic.py | 6 ++ 2 files changed, 7 insertions(+), 3 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-42958][CONNECT] Refactor `connect-jvm-client-mima-check` to support mima check with avro module
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 56cfd40e74d [SPARK-42958][CONNECT] Refactor `connect-jvm-client-mima-check` to support mima check with avro module 56cfd40e74d is described below commit 56cfd40e74d56362a425c5e5d5d9e7260176 Author: YangJie AuthorDate: Tue May 16 15:50:12 2023 -0400 [SPARK-42958][CONNECT] Refactor `connect-jvm-client-mima-check` to support mima check with avro module ### What changes were proposed in this pull request? This pr refactor `connect-jvm-client-mima-check` and `CheckConnectJvmClientCompatibility` to support mima check between `connect-client-jvm` and `avro` module. ### Why are the changes needed? Do mima check between `connect-client-jvm` and `avro` module. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions Closes #40605 from LuciferYang/SPARK-42958. Lead-authored-by: YangJie Co-authored-by: yangjie01 Signed-off-by: Herman van Hovell --- .../CheckConnectJvmClientCompatibility.scala | 109 +++-- dev/connect-jvm-client-mima-check | 10 +- 2 files changed, 83 insertions(+), 36 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index f9674ac38cd..ad99342e6e6 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -62,27 +62,25 @@ object CheckConnectJvmClientCompatibility { "spark-connect-client-jvm-assembly", "spark-connect-client-jvm") val sqlJar: File = findJar("sql/core", "spark-sql", "spark-sql") - val problems = checkMiMaCompatibility(clientJar, sqlJar) - if (problems.nonEmpty) { -resultWriter.write(s"ERROR: Comparing client jar: $clientJar and and sql jar: $sqlJar \n") -resultWriter.write(s"problems: \n") -resultWriter.write(s"${problems.map(p => p.description("client")).mkString("\n")}") -resultWriter.write("\n") -resultWriter.write( - "Exceptions to binary compatibility can be added in " + -"'CheckConnectJvmClientCompatibility#checkMiMaCompatibility'\n") - } + val problemsWithSqlModule = checkMiMaCompatibilityWithSqlModule(clientJar, sqlJar) + appendMimaCheckErrorMessageIfNeeded( +resultWriter, +problemsWithSqlModule, +clientJar, +sqlJar, +"Sql") + + val avroJar: File = findJar("connector/avro", "spark-avro", "spark-avro") + val problemsWithAvroModule = checkMiMaCompatibilityWithAvroModule(clientJar, sqlJar) + appendMimaCheckErrorMessageIfNeeded( +resultWriter, +problemsWithAvroModule, +clientJar, +avroJar, +"Avro") + val incompatibleApis = checkDatasetApiCompatibility(clientJar, sqlJar) - if (incompatibleApis.nonEmpty) { -resultWriter.write( - "ERROR: The Dataset apis only exist in the connect client " + -"module and not belong to the sql module include: \n") -resultWriter.write(incompatibleApis.mkString("\n")) -resultWriter.write("\n") -resultWriter.write( - "Exceptions can be added to exceptionMethods in " + - "'CheckConnectJvmClientCompatibility#checkDatasetApiCompatibility'\n") - } + appendIncompatibleDatasetApisErrorMessageIfNeeded(resultWriter, incompatibleApis) } catch { case e: Throwable => println(e.getMessage) @@ -94,16 +92,17 @@ object CheckConnectJvmClientCompatibility { } } - /** - * MiMa takes an old jar (sql jar) and a new jar (client jar) as inputs and then reports all - * incompatibilities found in the new jar. The incompatibility result is then filtered using - * include and exclude rules. Include rules are first applied to find all client classes that - * need to be checked. Then exclude rules are applied to filter out all unsupported methods in - * the client classes. - */ - private def checkMiMaCompatibility(clientJar: File, sqlJar: File): List[Problem] = { -val mima = new Mi
[spark] branch master updated: [SPARK-43223][CONNECT] Typed agg, reduce functions, RelationalGroupedDataset#as
This is an automated email from the ASF dual-hosted git repository. hvanhovell pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ba8cae2031f [SPARK-43223][CONNECT] Typed agg, reduce functions, RelationalGroupedDataset#as ba8cae2031f is described below commit ba8cae2031f81dc326d386cbe7d19c1f0a8f239e Author: Zhen Li AuthorDate: Mon May 15 11:05:33 2023 -0400 [SPARK-43223][CONNECT] Typed agg, reduce functions, RelationalGroupedDataset#as ### What changes were proposed in this pull request? Added the agg, reduce support in `KeyValueGroupedDataset`. Added `Dataset#reduce` Added `RelationalGroupedDataset#as`. Summary: * `KVGDS#agg`: `KVGDS#agg` and the `RelationalGroupedDS#agg` shares the exact same proto. The only difference is that the KVGDS always passing a UDF as the first grouping expression. That's also how we tell them apart in this PR. * `KVGDS#reduce`: Reduce is a special aggregation. The client uses an UnresolvedFunc "reduce" to mark the agg operator is a `ReduceAggregator` and calls `KVGDS#agg` directly. The server would be able to pick this func up directly and reuse the agg code path by sending in a `ReduceAggregator`. * `Dataset#reduce`: This is free after `KVGDS#reduce`. * `RelationalGroupedDS#as`: The only difference between `KVGDS` created using `ds#groupByKey` and `ds#agg#as` is the grouping expressions. The former requires one grouping func as the grouping expression, the latter uses a dummy func (to pass encoders/types to the server) + grouping expressions. Thus the server can count how many grouping expressions received and decide if the `KVGDS` should be created as `ds#groupByKey` or `ds#agg#as`. Followups: * [SPARK-43415] Support mapValues in the Agg functions. * [SPARK-43416] The tupled ProductEncoder dose not pick up the fields names from the server. ### Why are the changes needed? Missing APIs in Scala Client ### Does this PR introduce _any_ user-facing change? Added `KeyValueGrouppedDataset#agg, reduce`, `Dataset#reduce`, `RelationalGroupedDataset#as` methods for the Scala client. ### How was this patch tested? E2E tests Closes #40796 from zhenlineo/typed-agg. Authored-by: Zhen Li Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 66 +++-- .../apache/spark/sql/KeyValueGroupedDataset.scala | 255 -- .../spark/sql/RelationalGroupedDataset.scala | 14 +- .../sql/KeyValueGroupedDatasetE2ETestSuite.scala | 290 ++--- .../sql/UserDefinedFunctionE2ETestSuite.scala | 18 ++ .../CheckConnectJvmClientCompatibility.scala | 8 - .../spark/sql/connect/client/util/QueryTest.scala | 36 ++- .../apache/spark/sql/connect/common/UdfUtils.scala | 4 + .../sql/connect/planner/SparkConnectPlanner.scala | 209 +++ .../spark/sql/catalyst/plans/logical/object.scala | 16 ++ .../main/scala/org/apache/spark/sql/Column.scala | 13 +- .../apache/spark/sql/KeyValueGroupedDataset.scala | 15 +- .../spark/sql/RelationalGroupedDataset.scala | 53 ++-- .../spark/sql/expressions/ReduceAggregator.scala | 6 + .../apache/spark/sql/internal/TypedAggUtils.scala | 62 + 15 files changed, 883 insertions(+), 182 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 555f6c312c5..7a680bde7d3 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1242,10 +1242,7 @@ class Dataset[T] private[sql] ( */ @scala.annotation.varargs def groupBy(cols: Column*): RelationalGroupedDataset = { -new RelationalGroupedDataset( - toDF(), - cols.map(_.expr), - proto.Aggregate.GroupType.GROUP_TYPE_GROUPBY) +new RelationalGroupedDataset(toDF(), cols, proto.Aggregate.GroupType.GROUP_TYPE_GROUPBY) } /** @@ -1273,10 +1270,45 @@ class Dataset[T] private[sql] ( val colNames: Seq[String] = col1 +: cols new RelationalGroupedDataset( toDF(), - colNames.map(colName => Column(colName).expr), + colNames.map(colName => Column(colName)), proto.Aggregate.GroupType.GROUP_TYPE_GROUPBY) } + /** + * (Scala-specific) Reduces the elements of this Dataset using the specified binary function. + * The given `func` must be commutative and associative or the result may be non-deterministic. + * + * @group action + * @since 3.5.0 + */ + def reduce(func: (T, T) => T): T = { +val udf = ScalarUserDefinedFunction( + function = func, + inputEncoders = encoder :: encoder :: N