(spark) branch master updated: [SPARK-47818][CONNECT][FOLLOW-UP] Introduce plan cache in SparkConnectPlanner to improve performance of Analyze requests
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 51623785c38c [SPARK-47818][CONNECT][FOLLOW-UP] Introduce plan cache in SparkConnectPlanner to improve performance of Analyze requests 51623785c38c is described below commit 51623785c38c9b17a6d91cb8e7f686459bd4803e Author: Xi Lyu AuthorDate: Fri May 17 22:06:35 2024 +0200 [SPARK-47818][CONNECT][FOLLOW-UP] Introduce plan cache in SparkConnectPlanner to improve performance of Analyze requests ### What changes were proposed in this pull request? In [this previous PR](https://github.com/apache/spark/pull/46012), we introduced two new confs for the introduced plan cache - a static conf `spark.connect.session.planCache.maxSize` and a dynamic conf `spark.connect.session.planCache.enabled`. The plan cache is enabled by default with size 5. In this PR, we are marking them as internal because we don't expect users to deal with it. ### Why are the changes needed? These two confs are not expected to be used under normal circumstances, and we don't need to document them on the Spark Configuration reference page. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46638 from xi-db/SPARK-47818-plan-cache-followup2. Authored-by: Xi Lyu Signed-off-by: Herman van Hovell --- .../src/main/scala/org/apache/spark/sql/connect/config/Connect.scala| 2 ++ 1 file changed, 2 insertions(+) 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 f9d604712420..91f2e23a8b64 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 @@ -279,6 +279,7 @@ object Connect { .doc("Sets the maximum number of cached resolved logical plans in Spark Connect Session." + " If set to a value less or equal than zero will disable the plan cache.") .version("4.0.0") + .internal() .intConf .createWithDefault(5) @@ -289,6 +290,7 @@ object Connect { s" When false, the cache is disabled even if '${CONNECT_SESSION_PLAN_CACHE_SIZE.key}' is" + " greater than zero. The caching is best-effort and not guaranteed.") .version("4.0.0") + .internal() .booleanConf .createWithDefault(true) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
(spark) branch master updated: [SPARK-47545][CONNECT] Dataset `observe` support for the 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 21548a8cc5c5 [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client 21548a8cc5c5 is described below commit 21548a8cc5c527d4416a276a852f967b4410bd4b Author: Paddy Xu AuthorDate: Wed May 8 15:44:02 2024 -0400 [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client ### What changes were proposed in this pull request? This PR adds support for `Dataset.observe` to the Spark Connect Scala client. Note that the support here does not include listener support as it runs on the serve side. This PR includes a small refactoring to the `Observation` helper class. We extracted methods that are not bound to the SparkSession to `spark-api`, and added two subclasses on both `spark-core` and `spark-jvm-client`. ### Why are the changes needed? Before this PR, the `DF.observe` method is only supported in the Python client. ### Does this PR introduce _any_ user-facing change? Yes. The user can now issue `DF.observe(name, metrics...)` or `DF.observe(observationObject, metrics...)` to get stats of columns of a dataframe. ### How was this patch tested? Added new e2e tests. ### Was this patch authored or co-authored using generative AI tooling? Nope. Closes #45701 from xupefei/scala-observe. Authored-by: Paddy Xu Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 63 ++- .../scala/org/apache/spark/sql/Observation.scala | 46 + .../scala/org/apache/spark/sql/SparkSession.scala | 31 +++- .../org/apache/spark/sql/ClientE2ETestSuite.scala | 43 + .../CheckConnectJvmClientCompatibility.scala | 3 - .../src/main/protobuf/spark/connect/base.proto | 1 + .../spark/sql/connect/client/SparkResult.scala | 44 - .../common/LiteralValueProtoConverter.scala| 2 +- .../connect/execution/ExecuteThreadRunner.scala| 1 + .../execution/SparkConnectPlanExecution.scala | 12 +- python/pyspark/sql/connect/proto/base_pb2.py | 188 ++--- python/pyspark/sql/connect/proto/base_pb2.pyi | 5 +- .../org/apache/spark/sql/ObservationBase.scala | 113 + .../scala/org/apache/spark/sql/Observation.scala | 62 +-- 14 files changed, 448 insertions(+), 166 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 9a42afebf8f2..37f770319b69 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 @@ -3337,8 +3337,69 @@ class Dataset[T] private[sql] ( } } + /** + * Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset + * that returns the same result as the input, with the following guarantees: It will + * compute the defined aggregates (metrics) on all the data that is flowing through the Dataset + * at that point. It will report the value of the defined aggregate columns as soon as + * we reach a completion point. A completion point is currently defined as the end of a + * query. Please note that continuous execution is currently not supported. + * + * The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or + * more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that + * contain references to the input Dataset's columns must always be wrapped in an aggregate + * function. + * + * A user can retrieve the metrics by calling + * `org.apache.spark.sql.Dataset.collectResult().getObservedMetrics`. + * + * {{{ + * // Observe row count (rows) and highest id (maxid) in the Dataset while writing it + * val observed_ds = ds.observe("my_metrics", count(lit(1)).as("rows"), max($"id").as("maxid")) + * observed_ds.write.parquet("ds.parquet") + * val metrics = observed_ds.collectResult().getObservedMetrics + * }}} + * + * @group typedrel + * @since 4.0.0 + */ + @scala.annotation.varargs def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = { -throw new UnsupportedOperationException("observe is not implemented.") +sparkSession.newDataset(agnosticEncoder) { builder => + builder.getCollectMetricsBuilder +.setInput(plan.getRoot) +.setName(name) +.addAllMetrics((expr +: exprs).map(_.expr).asJava) +} + } + + /** + * Observe (named) metrics through an `org.ap
(spark) branch master updated: [SPARK-47818][CONNECT][FOLLOW-UP] Introduce plan cache in SparkConnectPlanner to improve performance of Analyze requests
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 675f5f0b599b [SPARK-47818][CONNECT][FOLLOW-UP] Introduce plan cache in SparkConnectPlanner to improve performance of Analyze requests 675f5f0b599b is described below commit 675f5f0b599ba650abed879ead677dd6cba818a2 Author: Xi Lyu AuthorDate: Fri Apr 26 13:15:21 2024 -0400 [SPARK-47818][CONNECT][FOLLOW-UP] Introduce plan cache in SparkConnectPlanner to improve performance of Analyze requests ### What changes were proposed in this pull request? In [the previous PR](https://github.com/apache/spark/pull/46012), we cache plans of AnalyzePlan requests. We're also enabling it for ExecutePlan in this PR. ### Why are the changes needed? Some operations like spark.sql() issue ExecutePlan requests. By caching them, we can also improve performance if subsequent plans to be analyzed include the plan returned by ExecutePlan as a subtree. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46098 from xi-db/SPARK-47818-plan-cache-followup. Authored-by: Xi Lyu Signed-off-by: Herman van Hovell --- .../sql/connect/execution/SparkConnectPlanExecution.scala | 2 +- .../sql/tests/connect/test_parity_memory_profiler.py | 15 +++ .../pyspark/sql/tests/connect/test_parity_udf_profiler.py | 15 +++ 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala index 32cdae7bae56..4f2b8c945127 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala @@ -71,7 +71,7 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder) val dataframe = Dataset.ofRows( sessionHolder.session, -planner.transformRelation(request.getPlan.getRoot), +planner.transformRelation(request.getPlan.getRoot, cachePlan = true), tracker, shuffleCleanupMode) responseObserver.onNext(createSchemaResponse(request.getSessionId, dataframe.schema)) diff --git a/python/pyspark/sql/tests/connect/test_parity_memory_profiler.py b/python/pyspark/sql/tests/connect/test_parity_memory_profiler.py index 513e49a144e5..c6ef9810c684 100644 --- a/python/pyspark/sql/tests/connect/test_parity_memory_profiler.py +++ b/python/pyspark/sql/tests/connect/test_parity_memory_profiler.py @@ -27,6 +27,20 @@ class MemoryProfilerParityTests(MemoryProfiler2TestsMixin, ReusedConnectTestCase super().setUp() self.spark._profiler_collector._value = None + +class MemoryProfilerWithoutPlanCacheParityTests(MemoryProfilerParityTests): +@classmethod +def setUpClass(cls): +super().setUpClass() +cls.spark.conf.set("spark.connect.session.planCache.enabled", False) + +@classmethod +def tearDownClass(cls): +try: +cls.spark.conf.unset("spark.connect.session.planCache.enabled") +finally: +super().tearDownClass() + def test_memory_profiler_udf_multiple_actions(self): def action(df): df.collect() @@ -35,6 +49,7 @@ class MemoryProfilerParityTests(MemoryProfiler2TestsMixin, ReusedConnectTestCase with self.sql_conf({"spark.sql.pyspark.udf.profiler": "memory"}): _do_computation(self.spark, action=action) +# Without the plan cache, UDF ID will be different for each action self.assertEqual(6, len(self.profile_results), str(list(self.profile_results))) for id in self.profile_results: diff --git a/python/pyspark/sql/tests/connect/test_parity_udf_profiler.py b/python/pyspark/sql/tests/connect/test_parity_udf_profiler.py index dfa56ff0bb88..a1789a50896d 100644 --- a/python/pyspark/sql/tests/connect/test_parity_udf_profiler.py +++ b/python/pyspark/sql/tests/connect/test_parity_udf_profiler.py @@ -27,6 +27,20 @@ class UDFProfilerParityTests(UDFProfiler2TestsMixin, ReusedConnectTestCase): super().setUp() self.spark._profiler_collector._value = None + +class UDFProfilerWithoutPlanCacheParityTests(UDFProfilerParityTests): +@classmethod +def setUpClass(cls): +super().setUpClass() +cls.spark.conf.set("spark.connect.session.planCache.
(spark) branch branch-3.5 updated: [SPARK-47819][CONNECT][3.5] Use asynchronous callback for execution cleanup
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 cdd077a39fd9 [SPARK-47819][CONNECT][3.5] Use asynchronous callback for execution cleanup cdd077a39fd9 is described below commit cdd077a39fd99ba7c2fba4e89f6ef9668cf3cbce Author: Xi Lyu AuthorDate: Wed Apr 24 09:08:59 2024 -0400 [SPARK-47819][CONNECT][3.5] Use asynchronous callback for execution cleanup ([Original PR](https://github.com/apache/spark/pull/46027)) ### What changes were proposed in this pull request? Expired sessions are regularly checked and cleaned up by a maintenance thread. However, currently, this process is synchronous. Therefore, in rare cases, interrupting the execution thread of a query in a session can take hours, causing the entire maintenance process to stall, resulting in a large amount of memory not being cleared. We address this by introducing asynchronous callbacks for execution cleanup, avoiding synchronous joins of execution threads, and preventing the maintenance thread from stalling in the above scenarios. To be more specific, instead of calling `runner.join()` in `ExecutorHolder.close()`, we set a post-cleanup function as the callback through `runner.processOnCompletion`, which will be called asynchronously once the execution runner is completed or interrupted. In this way, the maintenan [...] ### Why are the changes needed? In the rare cases mentioned above, performance can be severely affected. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests and a new test `Async cleanup callback gets called after the execution is closed` in `ReattachableExecuteSuite.scala`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46064 from xi-db/SPARK-47819-async-cleanup-3.5. Authored-by: Xi Lyu Signed-off-by: Herman van Hovell --- .../connect/execution/ExecuteThreadRunner.scala| 31 +- .../spark/sql/connect/service/ExecuteHolder.scala | 23 .../execution/ReattachableExecuteSuite.scala | 22 +++ .../connect/planner/SparkConnectServiceSuite.scala | 7 - 4 files changed, 64 insertions(+), 19 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala index 62083d4892f7..d503dde3d18c 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.connect.execution +import scala.concurrent.{ExecutionContext, Promise} +import scala.util.Try import scala.util.control.NonFatal import com.google.protobuf.Message @@ -29,7 +31,7 @@ import org.apache.spark.sql.connect.common.ProtoUtils import org.apache.spark.sql.connect.planner.SparkConnectPlanner import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteSessionTag} import org.apache.spark.sql.connect.utils.ErrorUtils -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * This class launches the actual execution in an execution thread. The execution pushes the @@ -37,10 +39,12 @@ import org.apache.spark.util.Utils */ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends Logging { + private val promise: Promise[Unit] = Promise[Unit]() + // The newly created thread will inherit all InheritableThreadLocals used by Spark, // e.g. SparkContext.localProperties. If considering implementing a thread-pool, // forwarding of thread locals needs to be taken into account. - private var executionThread: Thread = new ExecutionThread() + private val executionThread: Thread = new ExecutionThread(promise) private var interrupted: Boolean = false @@ -53,9 +57,11 @@ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends executionThread.start() } - /** Joins the background execution thread after it is finished. */ - def join(): Unit = { -executionThread.join() + /** + * Register a callback that gets executed after completion/interruption of the execution + */ + private[connect] def processOnCompletion(callback: Try[Unit] => Unit): Unit = { + promise.future.onComplete(callback)(ExecuteThreadRunner.namedExecutionContext) } /** @@ -222,10 +228,21 @@ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends .bu
(spark) branch master updated (4dad2170b05c -> 9b1b2b30d591)
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 4dad2170b05c [SPARK-47356][SQL] Add support for ConcatWs & Elt (all collations) add 9b1b2b30d591 [SPARK-47081][CONNECT][FOLLOW] Unflake Progress Execution No new revisions were added by this update. Summary of changes: .../sql/connect/execution/ExecuteGrpcResponseSender.scala | 2 -- .../sql/connect/execution/ExecuteResponseObserver.scala | 10 ++ .../spark/sql/connect/execution/ExecuteThreadRunner.scala | 5 +++-- python/pyspark/sql/connect/client/core.py | 10 +++--- python/pyspark/sql/tests/connect/shell/test_progress.py | 13 + 5 files changed, 29 insertions(+), 11 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-47694][CONNECT] Make max message size configurable on the client side
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 83fe9b16ab5a [SPARK-47694][CONNECT] Make max message size configurable on the client side 83fe9b16ab5a is described below commit 83fe9b16ab5a2eec5f844d1e30488fe48223e29b Author: Robert Dillitz AuthorDate: Mon Apr 15 14:52:52 2024 -0400 [SPARK-47694][CONNECT] Make max message size configurable on the client side ### What changes were proposed in this pull request? Follow up to https://github.com/apache/spark/pull/40447. Allows to configure the currently hardcoded max message of 128MB on the client side for both the Scala and Python clients. Adds the option to the Scala client and improves the way we handle `channelOptions` in Python's `ChannelBuiler`. ### Why are the changes needed? Usability - I am aware of two different cases where these limits are hit: 1. The user is trying to create a large dataframe from local data. We either hit the` grpc.max_send_message_length` in the Python client ([currently hardcoded](https://github.com/apache/spark/pull/40447/files)) or the `maxInboundMessageSize` on the cluster side ([now configurable](https://github.com/apache/spark/pull/40447/files)). 2. The result from the cluster has a single row that is larger than 128MB, causing an `ExecutePlanResponse` that is larger than the client's `grpc.max_receive_message_length` (Python) or `channel.maxInboundMessageSize` (Scala) ([both hardcoded](https://github.com/apache/spark/pull/40447/files)). This gives the option to increase these limits on the client side. ### Does this PR introduce _any_ user-facing change? Scala: Adds option to set `grpcMaxMessageSize` to `SparkConnectClient.Builder` Python: No. ### How was this patch tested? Tests added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45842 from dillitz/SPARK-47694. Authored-by: Robert Dillitz Signed-off-by: Herman van Hovell --- .../connect/client/SparkConnectClientSuite.scala | 9 +++- .../sql/connect/client/SparkConnectClient.scala| 14 +++-- .../connect/client/SparkConnectClientParser.scala | 24 +- python/pyspark/sql/connect/client/core.py | 20 +- .../sql/tests/connect/test_connect_session.py | 20 ++ 5 files changed, 69 insertions(+), 18 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala index 5a43cf014bdc..55f962b2a52c 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala @@ -310,7 +310,14 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { assert(client.userAgent.contains("scala/")) assert(client.userAgent.contains("jvm/")) assert(client.userAgent.contains("os/")) - })) + }), +TestPackURI( + "sc://SPARK-47694:123/;grpc_max_message_size=1860", + isCorrect = true, + client => { +assert(client.configuration.grpcMaxMessageSize == 1860) + }), +TestPackURI("sc://SPARK-47694:123/;grpc_max_message_size=abc", isCorrect = false)) private def checkTestPack(testPack: TestPackURI): Unit = { val client = SparkConnectClient.builder().connectionString(testPack.connectionString).build() diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index 746aaca6f559..d9d51c15a880 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -510,6 +510,7 @@ object SparkConnectClient { val PARAM_TOKEN = "token" val PARAM_USER_AGENT = "user_agent" val PARAM_SESSION_ID = "session_id" + val PARAM_GRPC_MAX_MESSAGE_SIZE = "grpc_max_message_size" } private def verifyURI(uri: URI): Unit = { @@ -558,6 +559,13 @@ object SparkConnectClient { def userAgent: String = _configuration.userAgent +def grpcMaxMessageSize(messageSize: Int): Builder = { + _configuration = _configuration.copy(grpcMaxMessageSize = messageSi
(spark) branch master updated: [SPARK-47819][CONNECT] Use asynchronous callback for execution cleanup
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 2bc9573e94f2 [SPARK-47819][CONNECT] Use asynchronous callback for execution cleanup 2bc9573e94f2 is described below commit 2bc9573e94f29cd5394429b623e30c4386a473ba Author: Xi Lyu AuthorDate: Fri Apr 12 08:48:40 2024 -0400 [SPARK-47819][CONNECT] Use asynchronous callback for execution cleanup ### What changes were proposed in this pull request? Expired sessions are regularly checked and cleaned up by a maintenance thread. However, currently, this process is synchronous. Therefore, in rare cases, interrupting the execution thread of a query in a session can take hours, causing the entire maintenance process to stall, resulting in a large amount of memory not being cleared. We address this by introducing asynchronous callbacks for execution cleanup, avoiding synchronous joins of execution threads, and preventing the maintenance thread from stalling in the above scenarios. To be more specific, instead of calling `runner.join()` in `ExecutorHolder.close()`, we set a post-cleanup function as the callback through `runner.processOnCompletion`, which will be called asynchronously once the execution runner is completed or interrupted. In this way, the maintenan [...] ### Why are the changes needed? In the rare cases mentioned above, performance can be severely affected. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests and a new test `Async cleanup callback gets called after the execution is closed` in `SparkConnectServiceE2ESuite.scala`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46027 from xi-db/SPARK-47819-async-cleanup. Authored-by: Xi Lyu Signed-off-by: Herman van Hovell --- .../connect/execution/ExecuteThreadRunner.scala| 33 -- .../spark/sql/connect/service/ExecuteHolder.scala | 16 --- .../connect/planner/SparkConnectServiceSuite.scala | 7 - .../service/SparkConnectServiceE2ESuite.scala | 23 +++ 4 files changed, 65 insertions(+), 14 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala index 56776819dac9..37c3120a8ff4 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala @@ -17,7 +17,9 @@ package org.apache.spark.sql.connect.execution +import scala.concurrent.{ExecutionContext, Promise} import scala.jdk.CollectionConverters._ +import scala.util.Try import scala.util.control.NonFatal import com.google.protobuf.Message @@ -30,7 +32,7 @@ import org.apache.spark.sql.connect.common.ProtoUtils import org.apache.spark.sql.connect.planner.SparkConnectPlanner import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteSessionTag, SparkConnectService} import org.apache.spark.sql.connect.utils.ErrorUtils -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * This class launches the actual execution in an execution thread. The execution pushes the @@ -38,10 +40,12 @@ import org.apache.spark.util.Utils */ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends Logging { + private val promise: Promise[Unit] = Promise[Unit]() + // The newly created thread will inherit all InheritableThreadLocals used by Spark, // e.g. SparkContext.localProperties. If considering implementing a thread-pool, // forwarding of thread locals needs to be taken into account. - private val executionThread: Thread = new ExecutionThread() + private val executionThread: ExecutionThread = new ExecutionThread(promise) private var started: Boolean = false @@ -63,11 +67,11 @@ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends } } - /** Joins the background execution thread after it is finished. */ - private[connect] def join(): Unit = { -// only called when the execution is completed or interrupted. -assert(completed || interrupted) -executionThread.join() + /** + * Register a callback that gets executed after completion/interruption of the execution thread. + */ + private[connect] def processOnCompletion(callback: Try[Unit] => Unit): Unit = { + promise.future.onComplete(callback)(ExecuteThreadRunner.namedExecutionContext) } /** @@ -276,10 +280,21 @@ private[conn
(spark) branch master updated: [SPARK-47380][CONNECT] Ensure on the server side that the SparkSession is the same
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 51e8634a5883 [SPARK-47380][CONNECT] Ensure on the server side that the SparkSession is the same 51e8634a5883 is described below commit 51e8634a5883f1816bb82c19b6e91c3516eee6c4 Author: Nemanja Boric AuthorDate: Mon Mar 18 15:44:29 2024 -0400 [SPARK-47380][CONNECT] Ensure on the server side that the SparkSession is the same ### What changes were proposed in this pull request? In this PR we change the client behaviour to send the previously observed server session id so that the server can validate that the client used to talk with this specific session. Previously this was only validated on the client side which made the server actually execute the request for the wrong session before throwing on the client side (once the response from the server was obtained). ### Why are the changes needed? The server can execute the client command on the wrong spark session before client figuring out it's the different session. ### Does this PR introduce _any_ user-facing change? The error message now pops up differently (it used to be a slightly different message when validated on the client). ### How was this patch tested? Existing unit tests, add new unit test, e2e test added, manual testing ### Was this patch authored or co-authored using generative AI tooling? No Closes #45499 from nemanja-boric-databricks/workspace. Authored-by: Nemanja Boric Signed-off-by: Herman van Hovell --- .../src/main/resources/error/error-classes.json| 5 + .../src/main/protobuf/spark/connect/base.proto | 54 .../sql/connect/client/ResponseValidator.scala | 16 +- .../sql/connect/client/SparkConnectClient.scala| 45 ++- .../service/SparkConnectAddArtifactsHandler.scala | 7 +- .../service/SparkConnectAnalyzeHandler.scala | 7 +- .../SparkConnectArtifactStatusesHandler.scala | 18 +- .../service/SparkConnectConfigHandler.scala| 9 +- .../service/SparkConnectExecutionManager.scala | 9 +- .../SparkConnectFetchErrorDetailsHandler.scala | 6 +- .../service/SparkConnectInterruptHandler.scala | 6 +- .../SparkConnectReattachExecuteHandler.scala | 8 +- .../SparkConnectReleaseExecuteHandler.scala| 4 +- .../SparkConnectReleaseSessionHandler.scala| 2 + .../sql/connect/service/SparkConnectService.scala | 9 +- .../service/SparkConnectSessionManager.scala | 29 +- .../execution/ReattachableExecuteSuite.scala | 8 +- .../connect/planner/SparkConnectServiceSuite.scala | 8 +- .../service/ArtifactStatusesHandlerSuite.scala | 1 + .../service/FetchErrorDetailsHandlerSuite.scala| 12 +- .../service/SparkConnectServiceE2ESuite.scala | 20 ++ .../service/SparkConnectSessionManagerSuite.scala | 38 ++- ...-error-conditions-invalid-handle-error-class.md | 4 + python/pyspark/sql/connect/client/core.py | 12 + python/pyspark/sql/connect/proto/base_pb2.py | 328 ++--- python/pyspark/sql/connect/proto/base_pb2.pyi | 210 + 26 files changed, 655 insertions(+), 220 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 9362b8342abf..b5a0089fb2c8 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -2083,6 +2083,11 @@ "Operation not found." ] }, + "SESSION_CHANGED" : { +"message" : [ + "The existing Spark server driver instance has restarted. Please reconnect." +] + }, "SESSION_CLOSED" : { "message" : [ "Session was closed." 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 cb9dbe62c193..bcc7edc0 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -66,6 +66,12 @@ message AnalyzePlanRequest { // The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff` string session_id = 1; + // (Optional) + // + // Server-side generated idempotency key from the previous responses (if any). Server + // can use this to validate that the server side session has not changed. + optional string client_observed_server_side_session_id = 17; + // (Required) User context UserContext user_context = 2; @@ -281,6 +287,12 @@ message ExecutePlanRequest
(spark) branch master updated: [SPARK-44815][CONNECT] Cache df.schema to avoid extra RPC
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 6f87fe2f513d [SPARK-44815][CONNECT] Cache df.schema to avoid extra RPC 6f87fe2f513d is described below commit 6f87fe2f513d1b1a022f0d03b6c81d73d7cfb228 Author: Martin Grund AuthorDate: Fri Feb 2 08:49:06 2024 -0400 [SPARK-44815][CONNECT] Cache df.schema to avoid extra RPC ### What changes were proposed in this pull request? This patch caches the result of the `df.schema` call in the DataFrame to avoid the extra roundtrip to the Spark Connect service to retrieve the columns or the schema. Since the Dataframe is immutable, the schema will not change. ### Why are the changes needed? Performance / Stability ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing UT Closes #42499 from grundprinzip/SPARK-44815. Lead-authored-by: Martin Grund Co-authored-by: Herman van Hovell Co-authored-by: Martin Grund Signed-off-by: Herman van Hovell --- .../jvm/src/main/scala/org/apache/spark/sql/Dataset.scala | 13 - python/pyspark/sql/connect/dataframe.py | 12 ++-- 2 files changed, 22 insertions(+), 3 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 949f53409386..9a42afebf8f2 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 @@ -244,7 +244,18 @@ class Dataset[T] private[sql] ( * @group basic * @since 3.4.0 */ - def schema: StructType = { + def schema: StructType = cachedSchema + + /** + * The cached schema. + * + * Schema caching is correct in most cases. Connect is lazy by nature. This means that we only + * resolve the plan when it is submitted for execution or analysis. We do not cache intermediate + * resolved plans. If the input (changes table, view redefinition, etc...) of the plan changes + * between the schema() call, and a subsequent action, the cached schema might be inconsistent + * with the end schema. + */ + private lazy val cachedSchema: StructType = { DataTypeProtoConverter .toCatalystType( sparkSession diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 6d37158142a6..4091016e0d59 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -120,6 +120,7 @@ class DataFrame: # Check whether _repr_html is supported or not, we use it to avoid calling RPC twice # by __repr__ and _repr_html_ while eager evaluation opens. self._support_repr_html = False +self._cached_schema: Optional[StructType] = None def __repr__(self) -> str: if not self._support_repr_html: @@ -1782,8 +1783,15 @@ class DataFrame: @property def schema(self) -> StructType: -query = self._plan.to_proto(self._session.client) -return self._session.client.schema(query) +# Schema caching is correct in most cases. Connect is lazy by nature. This means that +# we only resolve the plan when it is submitted for execution or analysis. We do not +# cache intermediate resolved plan. If the input (changes table, view redefinition, +# etc...) of the plan changes between the schema() call, and a subsequent action, the +# cached schema might be inconsistent with the end schema. +if self._cached_schema is None: +query = self._plan.to_proto(self._session.client) +self._cached_schema = self._session.client.schema(query) +return self._cached_schema schema.__doc__ = PySparkDataFrame.schema.__doc__ - 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-46600][SQL] Move shared code between SqlConf and SqlApiConf to SqlApiConfHelper
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 a753239ab1af [SPARK-46600][SQL] Move shared code between SqlConf and SqlApiConf to SqlApiConfHelper a753239ab1af is described below commit a753239ab1afaeddf4c991d42b93e4845f12e576 Author: Rui Wang AuthorDate: Mon Jan 8 22:22:06 2024 -0400 [SPARK-46600][SQL] Move shared code between SqlConf and SqlApiConf to SqlApiConfHelper ### What changes were proposed in this pull request? This code proposes to introduce a new object named `SqlApiConfHelper` to contain shared code between `SqlApiConf` and `SqlConf`. ### Why are the changes needed? As of now, SqlConf will access some of the variables of SqlApiConf while SqlApiConf also try to initialize SqlConf upon initialization. This PR is to avoid potential circular dependency between SqlConf and SqlApiConf. The shared variables or access to the shared variables are moved to the new `SqlApiConfHelper`. So either SqlApiConf and SqlConf wants to initialize the other side, they will only initialize the same third object. ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? Existing UT ### Was this patch authored or co-authored using generative AI tooling? NO Closes #44602 from amaliujia/refactor_sql_api. Authored-by: Rui Wang Signed-off-by: Herman van Hovell (cherry picked from commit 03fc5e26b866491b52f89f4d24beade7d1669a37) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/internal/SqlApiConf.scala | 26 .../spark/sql/internal/SqlApiConfHelper.scala | 48 ++ .../org/apache/spark/sql/internal/SQLConf.scala| 12 +++--- 3 files changed, 61 insertions(+), 25 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index d746e9037ec4..5ec72b83837e 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.internal import java.util.TimeZone -import java.util.concurrent.atomic.AtomicReference import scala.util.Try @@ -48,25 +47,14 @@ private[sql] trait SqlApiConf { private[sql] object SqlApiConf { // Shared keys. - val ANSI_ENABLED_KEY: String = "spark.sql.ansi.enabled" - val LEGACY_TIME_PARSER_POLICY_KEY: String = "spark.sql.legacy.timeParserPolicy" - val CASE_SENSITIVE_KEY: String = "spark.sql.caseSensitive" - val SESSION_LOCAL_TIMEZONE_KEY: String = "spark.sql.session.timeZone" - val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = "spark.sql.session.localRelationCacheThreshold" + val ANSI_ENABLED_KEY: String = SqlApiConfHelper.ANSI_ENABLED_KEY + val LEGACY_TIME_PARSER_POLICY_KEY: String = SqlApiConfHelper.LEGACY_TIME_PARSER_POLICY_KEY + val CASE_SENSITIVE_KEY: String = SqlApiConfHelper.CASE_SENSITIVE_KEY + val SESSION_LOCAL_TIMEZONE_KEY: String = SqlApiConfHelper.SESSION_LOCAL_TIMEZONE_KEY + val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = +SqlApiConfHelper.LOCAL_RELATION_CACHE_THRESHOLD_KEY - /** - * 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()() + def get: SqlApiConf = SqlApiConfHelper.getConfGetter.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$")) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala new file mode 100644 index ..79b6cb9231c5 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.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 req
(spark) branch master updated: [SPARK-46600][SQL] Move shared code between SqlConf and SqlApiConf to SqlApiConfHelper
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 03fc5e26b866 [SPARK-46600][SQL] Move shared code between SqlConf and SqlApiConf to SqlApiConfHelper 03fc5e26b866 is described below commit 03fc5e26b866491b52f89f4d24beade7d1669a37 Author: Rui Wang AuthorDate: Mon Jan 8 22:22:06 2024 -0400 [SPARK-46600][SQL] Move shared code between SqlConf and SqlApiConf to SqlApiConfHelper ### What changes were proposed in this pull request? This code proposes to introduce a new object named `SqlApiConfHelper` to contain shared code between `SqlApiConf` and `SqlConf`. ### Why are the changes needed? As of now, SqlConf will access some of the variables of SqlApiConf while SqlApiConf also try to initialize SqlConf upon initialization. This PR is to avoid potential circular dependency between SqlConf and SqlApiConf. The shared variables or access to the shared variables are moved to the new `SqlApiConfHelper`. So either SqlApiConf and SqlConf wants to initialize the other side, they will only initialize the same third object. ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? Existing UT ### Was this patch authored or co-authored using generative AI tooling? NO Closes #44602 from amaliujia/refactor_sql_api. Authored-by: Rui Wang Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/internal/SqlApiConf.scala | 26 .../spark/sql/internal/SqlApiConfHelper.scala | 48 ++ .../org/apache/spark/sql/internal/SQLConf.scala| 12 +++--- 3 files changed, 61 insertions(+), 25 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index d746e9037ec4..5ec72b83837e 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.internal import java.util.TimeZone -import java.util.concurrent.atomic.AtomicReference import scala.util.Try @@ -48,25 +47,14 @@ private[sql] trait SqlApiConf { private[sql] object SqlApiConf { // Shared keys. - val ANSI_ENABLED_KEY: String = "spark.sql.ansi.enabled" - val LEGACY_TIME_PARSER_POLICY_KEY: String = "spark.sql.legacy.timeParserPolicy" - val CASE_SENSITIVE_KEY: String = "spark.sql.caseSensitive" - val SESSION_LOCAL_TIMEZONE_KEY: String = "spark.sql.session.timeZone" - val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = "spark.sql.session.localRelationCacheThreshold" + val ANSI_ENABLED_KEY: String = SqlApiConfHelper.ANSI_ENABLED_KEY + val LEGACY_TIME_PARSER_POLICY_KEY: String = SqlApiConfHelper.LEGACY_TIME_PARSER_POLICY_KEY + val CASE_SENSITIVE_KEY: String = SqlApiConfHelper.CASE_SENSITIVE_KEY + val SESSION_LOCAL_TIMEZONE_KEY: String = SqlApiConfHelper.SESSION_LOCAL_TIMEZONE_KEY + val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = +SqlApiConfHelper.LOCAL_RELATION_CACHE_THRESHOLD_KEY - /** - * 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()() + def get: SqlApiConf = SqlApiConfHelper.getConfGetter.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$")) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala new file mode 100644 index ..79b6cb9231c5 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.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
(spark) branch master updated: [SPARK-46202][CONNECT] Expose new ArtifactManager APIs to support custom target directories
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 dcbebce9eacb [SPARK-46202][CONNECT] Expose new ArtifactManager APIs to support custom target directories dcbebce9eacb is described below commit dcbebce9eacb201cc8dfac918318be04ada842a8 Author: vicennial AuthorDate: Tue Dec 12 14:10:41 2023 -0800 [SPARK-46202][CONNECT] Expose new ArtifactManager APIs to support custom target directories ### What changes were proposed in this pull request? Adds new client APIs to the Spark Connect Scala Client: - `def addArtifact(bytes: Array[Byte], target: String): Unit` - `def addArtifact(source: String, target: String): Unit` ### Why are the changes needed? Currently, without the use of a REPL/Class finder, there is no API to support adding artifacts (file-based and in-memory) with a custom target directory structure to the remote Spark Connect session. ### Does this PR introduce _any_ user-facing change? Yes. Users can do the following for classfiles and jars: ```scala addArtifact("/Users/dummyUser/files/foo/bar.class", "sub/directory/foo.class") addArtifact(bytesBar, "bar.class") ``` This would preserve the directory structure in the remote server. In this case, the file would be stored under the directory: `$ROOT_SESSION_ARTIFACT_DIRECTORY/classes/bar.class` `$ROOT_SESSION_ARTIFACT_DIRECTORY/classes/sub/directory/foo.class` ### How was this patch tested? Unit test ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44109 from vicennial/SPARK-46202. Authored-by: vicennial Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 41 .../spark/sql/connect/client/ArtifactSuite.scala | 50 ++ .../spark/sql/connect/client/ArtifactManager.scala | 108 + .../sql/connect/client/SparkConnectClient.scala| 37 +++ 4 files changed, 219 insertions(+), 17 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 daa172e215ad..81c2ca11a7fb 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 @@ -590,6 +590,47 @@ class SparkSession private[sql] ( @Experimental def addArtifact(uri: URI): Unit = client.addArtifact(uri) + /** + * Add a single in-memory artifact to the session while preserving the directory structure + * specified by `target` under the session's working directory of that particular file + * extension. + * + * Supported target file extensions are .jar and .class. + * + * ==Example== + * {{{ + * addArtifact(bytesBar, "foo/bar.class") + * addArtifact(bytesFlat, "flat.class") + * // Directory structure of the session's working directory for class files would look like: + * // ${WORKING_DIR_FOR_CLASS_FILES}/flat.class + * // ${WORKING_DIR_FOR_CLASS_FILES}/foo/bar.class + * }}} + * + * @since 4.0.0 + */ + @Experimental + def addArtifact(bytes: Array[Byte], target: String): Unit = client.addArtifact(bytes, target) + + /** + * Add a single artifact to the session while preserving the directory structure specified by + * `target` under the session's working directory of that particular file extension. + * + * Supported target file extensions are .jar and .class. + * + * ==Example== + * {{{ + * addArtifact("/Users/dummyUser/files/foo/bar.class", "foo/bar.class") + * addArtifact("/Users/dummyUser/files/flat.class", "flat.class") + * // Directory structure of the session's working directory for class files would look like: + * // ${WORKING_DIR_FOR_CLASS_FILES}/flat.class + * // ${WORKING_DIR_FOR_CLASS_FILES}/foo/bar.class + * }}} + * + * @since 4.0.0 + */ + @Experimental + def addArtifact(source: String, target: String): Unit = client.addArtifact(source, target) + /** * Add one or more artifacts to the session. * diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/ArtifactSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/ArtifactSuite.scala index f945313d2427..0c8ef8e599fb 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/ArtifactSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/ArtifactSuite.scala
(spark) branch master updated: [SPARK-46265][CONNECT] Assertions in AddArtifact RPC make the connect client incompatible with older clusters
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 7c741c8c25fc [SPARK-46265][CONNECT] Assertions in AddArtifact RPC make the connect client incompatible with older clusters 7c741c8c25fc is described below commit 7c741c8c25fc4fe3d7d5fa5d490bb9b08debd952 Author: Niranjan Jayakar AuthorDate: Tue Dec 5 12:43:48 2023 -0400 [SPARK-46265][CONNECT] Assertions in AddArtifact RPC make the connect client incompatible with older clusters ### What changes were proposed in this pull request? A previous commit - d9c5f9d6 - updated the response of the `AddArtifact` RPC to return the session id. Further, it also added an assertion to the client of this RPC asserting that the session id returned by the server matches the session id that was requested. However, we will have the case that a connect client with this new assertion included may connect to a Spark cluster that does not yet have this change, making the change backwards incompatible. Loosen the assertion to allow empty session id in the RPC's response. ### Why are the changes needed? Newer connect clients can connect with older Spark clusters. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually started the a connect cluster using `spark-connect-shell` and using the connect client shell via `spark-connect-scala-client`. Changed the service to not return session id and verified the fix. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44181 from nija-at/artifact-sessionid. Authored-by: Niranjan Jayakar Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/connect/client/ArtifactManager.scala | 8 ++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala index 7a6eb963cb33..3cd35803d1ec 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala @@ -125,7 +125,9 @@ class ArtifactManager( .addAllNames(Arrays.asList(artifactName)) .build() val response = bstub.artifactStatus(request) -if (response.getSessionId != sessionId) { +if (StringUtils.isNotEmpty(response.getSessionId) && response.getSessionId != sessionId) { + // In older versions of the Spark cluster, the session ID is not set in the response. + // Ignore this check to keep compatibility. throw new IllegalStateException( s"Session ID mismatch: $sessionId != ${response.getSessionId}") } @@ -185,7 +187,9 @@ class ArtifactManager( val responseHandler = new StreamObserver[proto.AddArtifactsResponse] { private val summaries = mutable.Buffer.empty[ArtifactSummary] override def onNext(v: AddArtifactsResponse): Unit = { -if (v.getSessionId != sessionId) { +if (StringUtils.isNotEmpty(v.getSessionId) && v.getSessionId != sessionId) { + // In older versions of the Spark cluster, the session ID is not set in the response. + // Ignore this check to keep compatibility. throw new IllegalStateException(s"Session ID mismatch: $sessionId != ${v.getSessionId}") } v.getArtifactsList.forEach { summary => - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support
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 3e2b146eb81 [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support 3e2b146eb81 is described below commit 3e2b146eb81d9a5727f07b58f7bb1760a71a8697 Author: Vsevolod Stepanov AuthorDate: Wed Oct 25 21:35:07 2023 -0400 [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support ### What changes were proposed in this pull request? This PR enhances existing ClosureCleaner implementation to support cleaning closures defined in Ammonite. Please refer to [this gist](https://gist.github.com/vsevolodstep-db/b8e4d676745d6e2d047ecac291e5254c) to get more context on how Ammonite code wrapping works and what problems I'm trying to solve here. Overall, it contains these logical changes in `ClosureCleaner`: 1. Making it recognize and clean closures defined in Ammonite (previously it was checking if capturing class name starts with `$line` and ends with `$iw`, which is native Scala REPL specific thing 2. Making it clean closures if they are defined inside a user class in a REPL (see corner case 1 in the gist) 3. Making it clean nested closures properly for Ammonite REPL (see corner case 2 in the gist) 4. Making it transitively follow other Ammonite commands that are captured by the target closure. Please note that `cleanTransitively` option of `ClosureCleaner.clean()` method refers to following references transitively within enclosing command object, but it doesn't follow other command objects. As we need `ClosureCleaner` to be available in Spark Connect, I also moved the implementation to `common-utils` module. This brings a new `xbean-asm9-shaded` which is fairly small. Also, this PR moves `checkSerializable` check from `ClosureCleaner` to `SparkClosureCleaner`, as it is specific to Spark core The important changes affect `ClosureCleaner` only. They should not affect existing codepath for normal Scala closures / closures defined in a native Scala REPL and cover only closures defined in Ammonite. Also, this PR modifies SparkConnect's `UserDefinedFunction` to actually use `ClosureCleaner` and clean closures in SparkConnect ### Why are the changes needed? To properly support closures defined in Ammonite, reduce UDF payload size and avoid possible `NonSerializable` exceptions. This includes: - lambda capturing outer command object, leading in a circular dependency - lambda capturing other command objects transitively, exploding payload size ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. New tests in `ReplE2ESuite` covering various scenarios using SparkConnect + Ammonite REPL to make sure closures are actually cleaned. ### Was this patch authored or co-authored using generative AI tooling? No Closes #42995 from vsevolodstep-db/SPARK-45136/closure-cleaner. Authored-by: Vsevolod Stepanov Signed-off-by: Herman van Hovell --- common/utils/pom.xml | 4 + .../org/apache/spark/util/ClosureCleaner.scala | 636 ++--- .../org/apache/spark/util/SparkStreamUtils.scala | 109 .../sql/expressions/UserDefinedFunction.scala | 10 +- .../spark/sql/application/ReplE2ESuite.scala | 143 + .../CheckConnectJvmClientCompatibility.scala | 8 + core/pom.xml | 4 - .../main/scala/org/apache/spark/SparkContext.scala | 2 +- .../apache/spark/util/SparkClosureCleaner.scala| 49 ++ .../main/scala/org/apache/spark/util/Utils.scala | 85 +-- .../apache/spark/util/ClosureCleanerSuite.scala| 2 +- .../apache/spark/util/ClosureCleanerSuite2.scala | 4 +- project/MimaExcludes.scala | 4 +- .../catalyst/encoders/ExpressionEncoderSuite.scala | 4 +- .../org/apache/spark/streaming/StateSpec.scala | 6 +- 15 files changed, 756 insertions(+), 314 deletions(-) diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 37d1ea48d97..44cb30a19ff 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -39,6 +39,10 @@ org.apache.spark spark-tags_${scala.binary.version} + + org.apache.xbean + xbean-asm9-shaded + com.fasterxml.jackson.core jackson-databind diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/common/utils/src/main/scala/org/apache/spark/util/ClosureCleaner.scala similarity index 61% rename from core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala rename to common/utils/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 29fb0206f90..ffa2f0e60b2
[spark] branch master updated: [SPARK-44855][CONNECT] Small tweaks to attaching ExecuteGrpcResponseSender 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 292a1131b542 [SPARK-44855][CONNECT] Small tweaks to attaching ExecuteGrpcResponseSender to ExecuteResponseObserver 292a1131b542 is described below commit 292a1131b542ddc7b227a7e51e4f4233f3d2f9d8 Author: Juliusz Sompolski AuthorDate: Wed Oct 11 15:01:20 2023 -0400 [SPARK-44855][CONNECT] Small tweaks to attaching ExecuteGrpcResponseSender to ExecuteResponseObserver ### What changes were proposed in this pull request? Small improvements can be made to the way new ExecuteGrpcResponseSender is attached to observer. * Since now we have addGrpcResponseSender in ExecuteHolder, it should be ExecuteHolder responsibility to interrupt the old sender and that there is only one at a time, and to ExecuteResponseObserver's responsibility * executeObserver is used as a lock for synchronization. An explicit lock object could be better. Fix a small bug, when ExecuteGrpcResponseSender will not be waken up by interrupt if it was sleeping on the grpcCallObserverReadySignal. This would result in the sender potentially sleeping until the deadline (2 minutes) and only then removed, which would potentially delay timing the execution out by these 2 minutes. It should **not** cause any hang or wait on the client side, because if ExecuteGrpcResponseSender is interrupted, it means that the client has already came back with a ne [...] ### Why are the changes needed? Minor cleanup of previous work. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests in ReattachableExecuteSuite. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43181 from juliuszsompolski/SPARK-44855. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell --- .../execution/ExecuteGrpcResponseSender.scala | 26 - .../execution/ExecuteResponseObserver.scala| 44 ++ .../spark/sql/connect/service/ExecuteHolder.scala | 4 ++ 3 files changed, 40 insertions(+), 34 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala index 08496c36b28a..ba5ecc7a045a 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala @@ -63,15 +63,15 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( /** * Interrupt this sender and make it exit. */ - def interrupt(): Unit = executionObserver.synchronized { + def interrupt(): Unit = { interrupted = true -executionObserver.notifyAll() +wakeUp() } // For testing - private[connect] def setDeadline(deadlineMs: Long) = executionObserver.synchronized { + private[connect] def setDeadline(deadlineMs: Long) = { deadlineTimeMillis = deadlineMs -executionObserver.notifyAll() +wakeUp() } def run(lastConsumedStreamIndex: Long): Unit = { @@ -152,9 +152,6 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( s"lastConsumedStreamIndex=$lastConsumedStreamIndex") val startTime = System.nanoTime() -// register to be notified about available responses. -executionObserver.attachConsumer(this) - var nextIndex = lastConsumedStreamIndex + 1 var finished = false @@ -191,7 +188,7 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( sentResponsesSize > maximumResponseSize || deadlineTimeMillis < System.currentTimeMillis() logTrace(s"Trying to get next response with index=$nextIndex.") - executionObserver.synchronized { + executionObserver.responseLock.synchronized { logTrace(s"Acquired executionObserver lock.") val sleepStart = System.nanoTime() var sleepEnd = 0L @@ -208,7 +205,7 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( if (response.isEmpty) { val timeout = Math.max(1, deadlineTimeMillis - System.currentTimeMillis()) logTrace(s"Wait for response to become available with timeout=$timeout ms.") -executionObserver.wait(timeout) +executionObserver.responseLock.wait(timeout) logTrace(s"Reacquired executionObserver lock after waiting.") sleepEnd = System.nanoTime() } @@ -339,4 +336,
[spark] branch master updated (eae5c0e1efc -> 5ad57a70e51)
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 eae5c0e1efc [SPARK-45433][SQL] Fix CSV/JSON schema inference when timestamps do not match specified timestampFormat add 5ad57a70e51 [SPARK-45204][CONNECT] Add optional ExecuteHolder to SparkConnectPlanner No new revisions were added by this update. Summary of changes: .../connect/execution/ExecuteThreadRunner.scala| 7 +- .../execution/SparkConnectPlanExecution.scala | 2 +- .../sql/connect/planner/SparkConnectPlanner.scala | 84 ++ .../connect/planner/SparkConnectPlannerSuite.scala | 11 +-- .../plugin/SparkConnectPluginRegistrySuite.scala | 4 +- 5 files changed, 50 insertions(+), 58 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-45231][INFRA] Remove unrecognized and meaningless command about `Ammonite` from the GA testing workflow
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 5446f548bbc8 [SPARK-45231][INFRA] Remove unrecognized and meaningless command about `Ammonite` from the GA testing workflow 5446f548bbc8 is described below commit 5446f548bbc8a93414f1c773a8daf714b57b7d1a Author: panbingkun AuthorDate: Thu Oct 5 13:13:43 2023 -0400 [SPARK-45231][INFRA] Remove unrecognized and meaningless command about `Ammonite` from the GA testing workflow ### What changes were proposed in this pull request? The pr aims to remove unrecognized and meaningless command about `amm` from the GA testing workflow. ### Why are the changes needed? - When I observed GA's logs, I found the following logs: ``` Run # Fix for TTY related issues when launching the Ammonite REPL in tests. sh: 1: amm: not found ``` eg: https://github.com/apache/spark/actions/runs/6243934856/job/16950117287#step:10:21 https://github.com/panbingkun/spark/actions/runs/6232228999/job/16924063382#step:10:22 Obviously, `amm` did not recognize it. Through trial and error, it was found that the above command do not need to be executed in our GA. - Enhance maintainability and reduce misunderstandings. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42993 from panbingkun/remove_amm_test. Authored-by: panbingkun Signed-off-by: Herman van Hovell --- .github/workflows/build_and_test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 669cd76bd72f..5dce503a1799 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -261,7 +261,7 @@ jobs: shell: 'script -q -e -c "bash {0}"' run: | # Fix for TTY related issues when launching the Ammonite REPL in tests. -export TERM=vt100 && script -qfc 'echo exit | amm -s' && rm typescript +export TERM=vt100 # Hive "other tests" test needs larger metaspace size based on experiment. if [[ "$MODULES_TO_TEST" == "hive" ]] && [[ "$EXCLUDED_TAGS" == "org.apache.spark.tags.SlowHiveTest" ]]; then export METASPACE_SIZE=2g; fi export SERIAL_SBT_TESTS=1 - 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-45371][CONNECT] Fix shading issues in the 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 c5203abcbd1 [SPARK-45371][CONNECT] Fix shading issues in the Spark Connect Scala Client c5203abcbd1 is described below commit c5203abcbd191423071ef3603e95a7941bb1eec2 Author: Herman van Hovell AuthorDate: Mon Oct 2 13:03:06 2023 -0400 [SPARK-45371][CONNECT] Fix shading issues in the Spark Connect Scala Client ### What changes were proposed in this pull request? This PR fixes shading for the Spark Connect Scala Client maven build. The following things are addressed: - Guava and protobuf are included in the shaded jars. These were missing, and were causing users to see `ClassNotFoundException`s. - Fixed duplicate shading of guava. We use the parent pom's location now. - Fixed duplicate Netty dependency (shaded and transitive). One was used for GRPC and the other was needed by Arrow. This was fixed by pulling arrow into the shaded jar. - Use the same package as the shading defined in the parent package. ### Why are the changes needed? The maven artifacts for the Spark Connect Scala Client are currently broken. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual tests. Step 1: Build new shaded library and install it in local maven repository `build/mvn clean install -pl connector/connect/client/jvm -am -DskipTests` Step 2: Start Connect Server `connector/connect/bin/spark-connect` Step 3: Launch REPL using the newly created library This step requires [coursier](https://get-coursier.io/) to be installed. `cs launch --jvm zulu:17.0.8 --scala 2.13.9 -r m2Local com.lihaoyi:::ammonite:2.5.11 org.apache.spark::spark-connect-client-jvm:4.0.0-SNAPSHOT --java-opt --add-opens=java.base/java.nio=ALL-UNNAMED -M org.apache.spark.sql.application.ConnectRepl` Step 4: Run a bunch of commands: ```scala // Check version spark.version // Run a simple query { spark.range(1, 1, 1) .select($"id", $"id" % 5 as "group", rand(1).as("v1"), rand(2).as("v2")) .groupBy($"group") .agg( avg($"v1").as("v1_avg"), avg($"v2").as("v2_avg")) .show() } // Run a streaming query { import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger val query_name = "simple_streaming" val stream = spark.readStream .format("rate") .option("numPartitions", "1") .option("rowsPerSecond", "10") .load() .withWatermark("timestamp", "10 milliseconds") .groupBy(window(col("timestamp"), "10 milliseconds")) .count() .selectExpr("window.start as timestamp", "count as num_events") .writeStream .format("memory") .queryName(query_name) .trigger(ProcessingTimeTrigger.create("10 milliseconds")) // run for 20 seconds val query = stream.start() val start = System.currentTimeMillis() val end = System.currentTimeMillis() + 20 * 1000 while (System.currentTimeMillis() < end) { println(s"time: ${System.currentTimeMillis() - start} ms") println(query.status) spark.sql(s"select * from ${query_name}").show() Thread.sleep(500) } query.stop() } ``` Closes #43195 from hvanhovell/SPARK-45371. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit e53abbbceaa2c41babaa23fe4c2f282f559b4c03) Signed-off-by: Herman van Hovell --- connector/connect/client/jvm/pom.xml | 39 +++- 1 file changed, 30 insertions(+), 9 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 67227ef38eb..236e5850b76 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -50,10 +50,20 @@ spark-sketch_${scala.binary.version} ${project.version} + com.google.guava guava ${connect.guava.version} + compile + + + com.google.protobuf + protobuf-java + compile com.lihaoyi @@ -85,6 +95,7 @@ maven-shade-plugin false + true com.google.android:* @@ -92,52 +103,62 @@ com.google.code.findbugs:* com.google.code.gson:*
[spark] branch master updated: [SPARK-45371][CONNECT] Fix shading issues in the 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 e53abbbceaa [SPARK-45371][CONNECT] Fix shading issues in the Spark Connect Scala Client e53abbbceaa is described below commit e53abbbceaa2c41babaa23fe4c2f282f559b4c03 Author: Herman van Hovell AuthorDate: Mon Oct 2 13:03:06 2023 -0400 [SPARK-45371][CONNECT] Fix shading issues in the Spark Connect Scala Client ### What changes were proposed in this pull request? This PR fixes shading for the Spark Connect Scala Client maven build. The following things are addressed: - Guava and protobuf are included in the shaded jars. These were missing, and were causing users to see `ClassNotFoundException`s. - Fixed duplicate shading of guava. We use the parent pom's location now. - Fixed duplicate Netty dependency (shaded and transitive). One was used for GRPC and the other was needed by Arrow. This was fixed by pulling arrow into the shaded jar. - Use the same package as the shading defined in the parent package. ### Why are the changes needed? The maven artifacts for the Spark Connect Scala Client are currently broken. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual tests. Step 1: Build new shaded library and install it in local maven repository `build/mvn clean install -pl connector/connect/client/jvm -am -DskipTests` Step 2: Start Connect Server `connector/connect/bin/spark-connect` Step 3: Launch REPL using the newly created library This step requires [coursier](https://get-coursier.io/) to be installed. `cs launch --jvm zulu:17.0.8 --scala 2.13.9 -r m2Local com.lihaoyi:::ammonite:2.5.11 org.apache.spark::spark-connect-client-jvm:4.0.0-SNAPSHOT --java-opt --add-opens=java.base/java.nio=ALL-UNNAMED -M org.apache.spark.sql.application.ConnectRepl` Step 4: Run a bunch of commands: ```scala // Check version spark.version // Run a simple query { spark.range(1, 1, 1) .select($"id", $"id" % 5 as "group", rand(1).as("v1"), rand(2).as("v2")) .groupBy($"group") .agg( avg($"v1").as("v1_avg"), avg($"v2").as("v2_avg")) .show() } // Run a streaming query { import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger val query_name = "simple_streaming" val stream = spark.readStream .format("rate") .option("numPartitions", "1") .option("rowsPerSecond", "10") .load() .withWatermark("timestamp", "10 milliseconds") .groupBy(window(col("timestamp"), "10 milliseconds")) .count() .selectExpr("window.start as timestamp", "count as num_events") .writeStream .format("memory") .queryName(query_name) .trigger(ProcessingTimeTrigger.create("10 milliseconds")) // run for 20 seconds val query = stream.start() val start = System.currentTimeMillis() val end = System.currentTimeMillis() + 20 * 1000 while (System.currentTimeMillis() < end) { println(s"time: ${System.currentTimeMillis() - start} ms") println(query.status) spark.sql(s"select * from ${query_name}").show() Thread.sleep(500) } query.stop() } ``` Closes #43195 from hvanhovell/SPARK-45371. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- connector/connect/client/jvm/pom.xml | 39 +++- 1 file changed, 30 insertions(+), 9 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 9ca66b5c29c..a9040107f38 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -50,10 +50,20 @@ spark-sketch_${scala.binary.version} ${project.version} + com.google.guava guava ${connect.guava.version} + compile + + + com.google.protobuf + protobuf-java + compile com.lihaoyi @@ -85,6 +95,7 @@ maven-shade-plugin false + true com.google.android:* @@ -92,52 +103,62 @@ com.google.code.findbugs:* com.google.code.gson:* com.google.errorprone:* - com.google.guava:* com.google.j2objc:
[spark] branch branch-3.5 updated: [SPARK-45360][SQL][CONNECT] Initialize spark session builder configuration from SPARK_REMOTE
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 917bc8cb927 [SPARK-45360][SQL][CONNECT] Initialize spark session builder configuration from SPARK_REMOTE 917bc8cb927 is described below commit 917bc8cb92728267fb93891f4ef9da13c06e4589 Author: Yihong He AuthorDate: Thu Sep 28 12:58:07 2023 -0400 [SPARK-45360][SQL][CONNECT] Initialize spark session builder configuration from SPARK_REMOTE ### What changes were proposed in this pull request? - Initialize spark session builder configuration from SPARK_REMOTE ### Why are the changes needed? - `SparkSession.builder().getOrCreate()` should follow the behavior documents [here](https://github.com/apache/spark/blob/2cc1ee4d3a05a641d7a245f015ef824d8f7bae8b/docs/spark-connect-overview.md?plain=1#L241-L244) and support initialization from SPARK_REMOTE ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - `build/sbt "connect-client-jvm/testOnly *SparkConnectClientSuite"` ### Was this patch authored or co-authored using generative AI tooling? Closes #43153 from heyihong/SPARK-45360. Authored-by: Yihong He Signed-off-by: Herman van Hovell (cherry picked from commit 183a3d761f36d35572cfb37ab921b6a86f8f28ed) Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 5 +- .../connect/client/SparkConnectClientSuite.scala | 61 ++ 2 files changed, 65 insertions(+), 1 deletion(-) 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 7bd8fa59aea..421f37b9e8a 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 @@ -783,7 +783,10 @@ object SparkSession extends Logging { } class Builder() extends Logging { -private val builder = SparkConnectClient.builder() +// Initialize the connection string of the Spark Connect client builder from SPARK_REMOTE +// by default, if it exists. The connection string can be overridden using +// the remote() function, as it takes precedence over the SPARK_REMOTE environment variable. +private val builder = SparkConnectClient.builder().loadFromEnvironment() private var client: SparkConnectClient = _ private[this] val options = new scala.collection.mutable.HashMap[String, String] diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala index 80e245ec78b..89acc2c60ac 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala @@ -86,6 +86,24 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { assert(response.getSessionId === "abc123") } + private def withEnvs(pairs: (String, String)*)(f: => Unit): Unit = { +val readonlyEnv = System.getenv() +val field = readonlyEnv.getClass.getDeclaredField("m") +field.setAccessible(true) +val modifiableEnv = field.get(readonlyEnv).asInstanceOf[java.util.Map[String, String]] +try { + for ((k, v) <- pairs) { +assert(!modifiableEnv.containsKey(k)) +modifiableEnv.put(k, v) + } + f +} finally { + for ((k, _) <- pairs) { +modifiableEnv.remove(k) + } +} + } + test("Test connection") { testClientConnection() { testPort => SparkConnectClient.builder().port(testPort).build() } } @@ -112,6 +130,49 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { } } + test("SparkSession create with SPARK_REMOTE") { +startDummyServer(0) + +withEnvs("SPARK_REMOTE" -> s"sc://localhost:${server.getPort}") { + val session = SparkSession.builder().create() + val df = session.range(10) + df.analyze // Trigger RPC + assert(df.plan === service.getAndClearLatestInputPlan()) + + val session2 = SparkSession.builder().create() + assert(session != session2) +} + } + + test("SparkSession getOrCreate with SPARK_REMOTE") { +startDummyServer(0) + +withEnvs("SPARK_REMOTE" -> s"sc://localhost:${server.getPort}") { + val session = S
[spark] branch master updated: [SPARK-45360][SQL][CONNECT] Initialize spark session builder configuration from SPARK_REMOTE
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 183a3d761f3 [SPARK-45360][SQL][CONNECT] Initialize spark session builder configuration from SPARK_REMOTE 183a3d761f3 is described below commit 183a3d761f36d35572cfb37ab921b6a86f8f28ed Author: Yihong He AuthorDate: Thu Sep 28 12:58:07 2023 -0400 [SPARK-45360][SQL][CONNECT] Initialize spark session builder configuration from SPARK_REMOTE ### What changes were proposed in this pull request? - Initialize spark session builder configuration from SPARK_REMOTE ### Why are the changes needed? - `SparkSession.builder().getOrCreate()` should follow the behavior documents [here](https://github.com/apache/spark/blob/2cc1ee4d3a05a641d7a245f015ef824d8f7bae8b/docs/spark-connect-overview.md?plain=1#L241-L244) and support initialization from SPARK_REMOTE ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - `build/sbt "connect-client-jvm/testOnly *SparkConnectClientSuite"` ### Was this patch authored or co-authored using generative AI tooling? Closes #43153 from heyihong/SPARK-45360. Authored-by: Yihong He Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/SparkSession.scala | 5 +- .../connect/client/SparkConnectClientSuite.scala | 61 ++ 2 files changed, 65 insertions(+), 1 deletion(-) 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 abe4d6a96e6..42052e3f8e6 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 @@ -786,7 +786,10 @@ object SparkSession extends Logging { } class Builder() extends Logging { -private val builder = SparkConnectClient.builder() +// Initialize the connection string of the Spark Connect client builder from SPARK_REMOTE +// by default, if it exists. The connection string can be overridden using +// the remote() function, as it takes precedence over the SPARK_REMOTE environment variable. +private val builder = SparkConnectClient.builder().loadFromEnvironment() private var client: SparkConnectClient = _ private[this] val options = new scala.collection.mutable.HashMap[String, String] diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala index 488118d0552..57e0b4016f1 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala @@ -86,6 +86,24 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { assert(response.getSessionId === "abc123") } + private def withEnvs(pairs: (String, String)*)(f: => Unit): Unit = { +val readonlyEnv = System.getenv() +val field = readonlyEnv.getClass.getDeclaredField("m") +field.setAccessible(true) +val modifiableEnv = field.get(readonlyEnv).asInstanceOf[java.util.Map[String, String]] +try { + for ((k, v) <- pairs) { +assert(!modifiableEnv.containsKey(k)) +modifiableEnv.put(k, v) + } + f +} finally { + for ((k, _) <- pairs) { +modifiableEnv.remove(k) + } +} + } + test("Test connection") { testClientConnection() { testPort => SparkConnectClient.builder().port(testPort).build() } } @@ -112,6 +130,49 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { } } + test("SparkSession create with SPARK_REMOTE") { +startDummyServer(0) + +withEnvs("SPARK_REMOTE" -> s"sc://localhost:${server.getPort}") { + val session = SparkSession.builder().create() + val df = session.range(10) + df.analyze // Trigger RPC + assert(df.plan === service.getAndClearLatestInputPlan()) + + val session2 = SparkSession.builder().create() + assert(session != session2) +} + } + + test("SparkSession getOrCreate with SPARK_REMOTE") { +startDummyServer(0) + +withEnvs("SPARK_REMOTE" -> s"sc://localhost:${server.getPort}") { + val session = SparkSession.builder().getOrCreate() + + val df = session.range(10) + df.analyze // Trigger RPC +
[spark] branch master updated: [SPARK-44622][SQL][CONNECT] Implement FetchErrorDetails RPC
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 48b1a283a2eb [SPARK-44622][SQL][CONNECT] Implement FetchErrorDetails RPC 48b1a283a2eb is described below commit 48b1a283a2eba9f70149d5980d074fad2743c4ff Author: Yihong He AuthorDate: Wed Sep 20 00:14:44 2023 -0400 [SPARK-44622][SQL][CONNECT] Implement FetchErrorDetails RPC ### What changes were proposed in this pull request? - Introduced the FetchErrorDetails RPC to retrieve comprehensive error details. FetchErrorDetails is used for enriching the error by issuing a separate RPC call based on the `errorId` field in the ErrorInfo. - Introduced error enrichment that utilizes an additional RPC to fetch untruncated exception messages and server-side stack traces. This enrichment can be enabled or disabled using the flag `spark.sql.connect.enrichError.enabled`, and it's true by default. - Implemented setting server-side stack traces for exceptions on the client side via FetchErrorDetails RPC for debugging. The feature is enabled or disabled using the flag `spark.sql.connect.serverStacktrace.enabled` and it's true by default ### Why are the changes needed? - Attaching full exception messages to the error details protobuf can quickly hit the 8K GRPC Netty header limit. Utilizing a separate RPC to fetch comprehensive error information is more dependable. - Providing server-side stack traces aids in effectively diagnosing server-related issues. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - `build/sbt "connect/testOnly *FetchErrorDetailsHandlerSuite"` ### Was this patch authored or co-authored using generative AI tooling? No Closes #42377 from heyihong/SPARK-44622. Authored-by: Yihong He Signed-off-by: Herman van Hovell --- .../src/main/protobuf/spark/connect/base.proto | 57 +++ .../apache/spark/sql/connect/config/Connect.scala | 18 +++ .../spark/sql/connect/service/SessionHolder.scala | 21 ++- .../SparkConnectFetchErrorDetailsHandler.scala | 59 +++ .../sql/connect/service/SparkConnectService.scala | 14 ++ .../spark/sql/connect/utils/ErrorUtils.scala | 103 ++-- .../service/FetchErrorDetailsHandlerSuite.scala| 166 +++ python/pyspark/sql/connect/proto/base_pb2.py | 14 +- python/pyspark/sql/connect/proto/base_pb2.pyi | 180 + python/pyspark/sql/connect/proto/base_pb2_grpc.py | 45 ++ 10 files changed, 659 insertions(+), 18 deletions(-) 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 65e2493f8368..cf1355f7ebc1 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -778,6 +778,60 @@ message ReleaseExecuteResponse { optional string operation_id = 2; } +message FetchErrorDetailsRequest { + + // (Required) + // The session_id specifies a Spark session for a user identified by user_context.user_id. + // The id should be a UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`. + string session_id = 1; + + // User context + UserContext user_context = 2; + + // (Required) + // The id of the error. + string error_id = 3; +} + +message FetchErrorDetailsResponse { + + message StackTraceElement { +// The fully qualified name of the class containing the execution point. +string declaring_class = 1; + +// The name of the method containing the execution point. +string method_name = 2; + +// The name of the file containing the execution point. +string file_name = 3; + +// The line number of the source line containing the execution point. +int32 line_number = 4; + } + + // Error defines the schema for the representing exception. + message Error { +// The fully qualified names of the exception class and its parent classes. +repeated string error_type_hierarchy = 1; + +// The detailed message of the exception. +string message = 2; + +// The stackTrace of the exception. It will be set +// if the SQLConf spark.sql.connect.serverStacktrace.enabled is true. +repeated StackTraceElement stack_trace = 3; + +// The index of the cause error in errors. +optional int32 cause_idx = 4; + } + + // The index of the root error in errors. The field will not be set if the error is not found. + optional int32 root_error_idx = 1; + + // A list of errors. + repeated Error errors = 2; +} + // Main interface for the SparkConnect service. service SparkConnectService { @@ -813,5 +867,8
[spark] branch branch-3.5 updated: [SPARK-44872][CONNECT][FOLLOWUP] Deflake ReattachableExecuteSuite and increase retry buffer
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 b2aead9f98d [SPARK-44872][CONNECT][FOLLOWUP] Deflake ReattachableExecuteSuite and increase retry buffer b2aead9f98d is described below commit b2aead9f98d900d139cff41d53f79a37e1e09e81 Author: Juliusz Sompolski AuthorDate: Fri Sep 15 19:06:58 2023 -0700 [SPARK-44872][CONNECT][FOLLOWUP] Deflake ReattachableExecuteSuite and increase retry buffer ### What changes were proposed in this pull request? Deflake tests in ReattachableExecuteSuite and increase CONNECT_EXECUTE_REATTACHABLE_OBSERVER_RETRY_BUFFER_SIZE. ### Why are the changes needed? Two tests could be flaky with errors `INVALID_CURSOR.POSITION_NOT_AVAILABLE`. This is caused when a server releases the response when it falls more than CONNECT_EXECUTE_REATTACHABLE_OBSERVER_RETRY_BUFFER_SIZE behind the latest response it sent. However, because of HTTP2 flow control, the responses could still be in transit. In the test suite, we were explicitly disconnecting the iterators and later reconnect... In some cases they could not reconnect, because the response they last seen have fallen too fare behind. This not only changes the suite, but also adjust the default config. This potentially makes the reconnecting more robust. In normal situation, it should not lead to increased memory pressure, because the clients also release the responses using ReleaseExecute as soon as they are received. Normally, buffered responses should be freed by ReleaseExecute and this retry buffer is only a fallback mechanism. Therefore, it is safe to increase the default. In practice, this would only have effect in cases where there are actual network errors, and the increased buffer size should make the reconnects more robust in these cases. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? ReattachableExecuteSuite. Did more manual experiments of how far the response sent by client can be behind the response sent by server (because of HTTP2 flow control window) ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42908 from juliuszsompolski/SPARK-44872-followup. Authored-by: Juliusz Sompolski Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/connect/config/Connect.scala | 2 +- .../spark/sql/connect/SparkConnectServerTest.scala | 2 +- .../execution/ReattachableExecuteSuite.scala | 26 +- 3 files changed, 18 insertions(+), 12 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 7b8b05ce11a..253ac38f9cf 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 @@ -133,7 +133,7 @@ object Connect { "With any value greater than 0, the last sent response will always be buffered.") .version("3.5.0") .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("1m") + .createWithDefaultString("10m") val CONNECT_EXTENSIONS_RELATION_CLASSES = buildStaticConf("spark.connect.extensions.relation.classes") diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala index 488858d33ea..eddd1c6be72 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.test.SharedSparkSession * Base class and utilities for a test suite that starts and tests the real SparkConnectService * with a real SparkConnectClient, communicating over RPC, but both in-process. */ -class SparkConnectServerTest extends SharedSparkSession { +trait SparkConnectServerTest extends SharedSparkSession { // Server port val serverPort: Int = diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala index 169b15582b6..0e29a07b719 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala +++ b/connector/connect/server/src
[spark] branch branch-3.5 updated: [SPARK-44872][CONNECT] Server testing infra and ReattachableExecuteSuite
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 af8c0b999be [SPARK-44872][CONNECT] Server testing infra and ReattachableExecuteSuite af8c0b999be is described below commit af8c0b999be746b661efe2439ac015a0c7d12c00 Author: Juliusz Sompolski AuthorDate: Tue Sep 12 16:48:26 2023 +0200 [SPARK-44872][CONNECT] Server testing infra and ReattachableExecuteSuite ### What changes were proposed in this pull request? Add `SparkConnectServerTest` with infra to test real server with real client in the same process, but communicating over RPC. Add `ReattachableExecuteSuite` with some tests for reattachable execute. Two bugs were found by the tests: * Fix bug in `SparkConnectExecutionManager.createExecuteHolder` when attempting to resubmit an operation that was deemed abandoned. This bug is benign in reattachable execute, because reattachable execute would first send a ReattachExecute, which would be handled correctly in SparkConnectReattachExecuteHandler. For non-reattachable execute (disabled or old client), this is also a very unlikely scenario, because the retrying mechanism should be able to resubmit before the query is decl [...] * In `ExecuteGrpcResponseSender` there was an assertion that assumed that if `sendResponse` did not send, it was because deadline was reached. But it can also be because of interrupt. This would have resulted in interrupt returning an assertion error instead of CURSOR_DISCONNECTED in testing. Outside of testing assertions are not enabled, so this was not a problem outside of testing. ### Why are the changes needed? Testing of reattachable execute. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Tests added. Closes #42560 from juliuszsompolski/sc-reattachable-tests. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell (cherry picked from commit 4b96add471d292ed5c63ccc625489ff78cfb9b25) Signed-off-by: Herman van Hovell --- .../sql/connect/client/CloseableIterator.scala | 22 +- .../client/CustomSparkConnectBlockingStub.scala| 2 +- .../ExecutePlanResponseReattachableIterator.scala | 18 +- .../connect/client/GrpcExceptionConverter.scala| 5 +- .../sql/connect/client/GrpcRetryHandler.scala | 4 +- .../execution/ExecuteGrpcResponseSender.scala | 17 +- .../execution/ExecuteResponseObserver.scala| 8 +- .../spark/sql/connect/service/ExecuteHolder.scala | 10 + .../service/SparkConnectExecutionManager.scala | 40 ++- .../spark/sql/connect/SparkConnectServerTest.scala | 261 +++ .../execution/ReattachableExecuteSuite.scala | 352 + .../scala/org/apache/spark/SparkFunSuite.scala | 24 ++ 12 files changed, 735 insertions(+), 28 deletions(-) diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala index 891e50ed6e7..d3fc9963edc 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala @@ -27,6 +27,20 @@ private[sql] trait CloseableIterator[E] extends Iterator[E] with AutoCloseable { } } +private[sql] abstract class WrappedCloseableIterator[E] extends CloseableIterator[E] { + + def innerIterator: Iterator[E] + + override def next(): E = innerIterator.next() + + override def hasNext(): Boolean = innerIterator.hasNext + + override def close(): Unit = innerIterator match { +case it: CloseableIterator[E] => it.close() +case _ => // nothing + } +} + private[sql] object CloseableIterator { /** @@ -35,12 +49,8 @@ private[sql] object CloseableIterator { def apply[T](iterator: Iterator[T]): CloseableIterator[T] = iterator match { case closeable: CloseableIterator[T] => closeable case _ => - new CloseableIterator[T] { -override def next(): T = iterator.next() - -override def hasNext(): Boolean = iterator.hasNext - -override def close() = { /* empty */ } + new WrappedCloseableIterator[T] { +override def innerIterator = iterator } } } diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala index 73ff01e223f..80edcfa8be1 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/co
[spark] branch master updated (d8298bffd91 -> 4b96add471d)
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 d8298bffd91 [SPARK-45081][SQL] Encoders.bean does no longer work with read-only properties add 4b96add471d [SPARK-44872][CONNECT] Server testing infra and ReattachableExecuteSuite No new revisions were added by this update. Summary of changes: .../sql/connect/client/CloseableIterator.scala | 22 +- .../client/CustomSparkConnectBlockingStub.scala| 2 +- .../ExecutePlanResponseReattachableIterator.scala | 18 +- .../connect/client/GrpcExceptionConverter.scala| 5 +- .../sql/connect/client/GrpcRetryHandler.scala | 4 +- .../execution/ExecuteGrpcResponseSender.scala | 17 +- .../execution/ExecuteResponseObserver.scala| 8 +- .../spark/sql/connect/service/ExecuteHolder.scala | 10 + .../service/SparkConnectExecutionManager.scala | 40 ++- .../spark/sql/connect/SparkConnectServerTest.scala | 261 +++ .../execution/ReattachableExecuteSuite.scala | 352 + .../scala/org/apache/spark/SparkFunSuite.scala | 24 ++ 12 files changed, 735 insertions(+), 28 deletions(-) create mode 100644 connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala create mode 100644 connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.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-45081][SQL] Encoders.bean does no longer work with read-only properties
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 ffa4127c774 [SPARK-45081][SQL] Encoders.bean does no longer work with read-only properties ffa4127c774 is described below commit ffa4127c774ea13b4d6bbcc82bc5a9bee23d7156 Author: Giambattista Bloisi AuthorDate: Tue Sep 12 16:16:04 2023 +0200 [SPARK-45081][SQL] Encoders.bean does no longer work with read-only properties ### What changes were proposed in this pull request? This PR re-enables Encoders.bean to be called against beans having read-only properties, that is properties that have only getters and no setter method. Beans with read only properties are even used in internal tests. Setter methods of a Java bean encoder are stored within an Option wrapper because they are missing in case of read-only properties. When a java bean has to be initialized, setter methods for the bean properties have to be called: this PR filters out read-only properties from that process. ### Why are the changes needed? The changes are required to avoid an exception to the thrown by getting the value of a None option object. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? An additional regression test has been added ### Was this patch authored or co-authored using generative AI tooling? No Closes #42829 from gbloisi-openaire/SPARK-45081. Authored-by: Giambattista Bloisi Signed-off-by: Herman van Hovell (cherry picked from commit d8298bffd91de01299f9456b37e4454e8b4a6ae8) Signed-off-by: Herman van Hovell --- .../sql/connect/client/arrow/ArrowDeserializer.scala | 20 +++- .../spark/sql/catalyst/DeserializerBuildHelper.scala | 4 +++- .../test/org/apache/spark/sql/JavaDatasetSuite.java | 17 + 3 files changed, 31 insertions(+), 10 deletions(-) diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala index cd54966ccf5..94295785987 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala @@ -332,15 +332,17 @@ object ArrowDeserializers { val constructor = methodLookup.findConstructor(tag.runtimeClass, MethodType.methodType(classOf[Unit])) val lookup = createFieldLookup(vectors) -val setters = fields.map { field => - val vector = lookup(field.name) - val deserializer = deserializerFor(field.enc, vector, timeZoneId) - val setter = methodLookup.findVirtual( -tag.runtimeClass, -field.writeMethod.get, -MethodType.methodType(classOf[Unit], field.enc.clsTag.runtimeClass)) - (bean: Any, i: Int) => setter.invoke(bean, deserializer.get(i)) -} +val setters = fields + .filter(_.writeMethod.isDefined) + .map { field => +val vector = lookup(field.name) +val deserializer = deserializerFor(field.enc, vector, timeZoneId) +val setter = methodLookup.findVirtual( + tag.runtimeClass, + field.writeMethod.get, + MethodType.methodType(classOf[Unit], field.enc.clsTag.runtimeClass)) +(bean: Any, i: Int) => setter.invoke(bean, deserializer.get(i)) + } new StructFieldSerializer[Any](struct) { def value(i: Int): Any = { val instance = constructor.invoke() 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 16a7d7ff065..0b88d5a4130 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 @@ -390,7 +390,9 @@ object DeserializerBuildHelper { CreateExternalRow(convertedFields, enc.schema)) case JavaBeanEncoder(tag, fields) => - val setters = fields.map { f => + val setters = fields +.filter(_.writeMethod.isDefined) +.map { f => val newTypePath = walkedTypePath.recordField( f.enc.clsTag.runtimeClass.getName, f.name) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index 4f7cf8da787..f416d411322 100644 --- a/sql/core/src/test
[spark] branch master updated: [SPARK-45081][SQL] Encoders.bean does no longer work with read-only properties
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 d8298bffd91 [SPARK-45081][SQL] Encoders.bean does no longer work with read-only properties d8298bffd91 is described below commit d8298bffd91de01299f9456b37e4454e8b4a6ae8 Author: Giambattista Bloisi AuthorDate: Tue Sep 12 16:16:04 2023 +0200 [SPARK-45081][SQL] Encoders.bean does no longer work with read-only properties ### What changes were proposed in this pull request? This PR re-enables Encoders.bean to be called against beans having read-only properties, that is properties that have only getters and no setter method. Beans with read only properties are even used in internal tests. Setter methods of a Java bean encoder are stored within an Option wrapper because they are missing in case of read-only properties. When a java bean has to be initialized, setter methods for the bean properties have to be called: this PR filters out read-only properties from that process. ### Why are the changes needed? The changes are required to avoid an exception to the thrown by getting the value of a None option object. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? An additional regression test has been added ### Was this patch authored or co-authored using generative AI tooling? No Closes #42829 from gbloisi-openaire/SPARK-45081. Authored-by: Giambattista Bloisi Signed-off-by: Herman van Hovell --- .../sql/connect/client/arrow/ArrowDeserializer.scala | 20 +++- .../spark/sql/catalyst/DeserializerBuildHelper.scala | 4 +++- .../test/org/apache/spark/sql/JavaDatasetSuite.java | 17 + 3 files changed, 31 insertions(+), 10 deletions(-) diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala index cd54966ccf5..94295785987 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala @@ -332,15 +332,17 @@ object ArrowDeserializers { val constructor = methodLookup.findConstructor(tag.runtimeClass, MethodType.methodType(classOf[Unit])) val lookup = createFieldLookup(vectors) -val setters = fields.map { field => - val vector = lookup(field.name) - val deserializer = deserializerFor(field.enc, vector, timeZoneId) - val setter = methodLookup.findVirtual( -tag.runtimeClass, -field.writeMethod.get, -MethodType.methodType(classOf[Unit], field.enc.clsTag.runtimeClass)) - (bean: Any, i: Int) => setter.invoke(bean, deserializer.get(i)) -} +val setters = fields + .filter(_.writeMethod.isDefined) + .map { field => +val vector = lookup(field.name) +val deserializer = deserializerFor(field.enc, vector, timeZoneId) +val setter = methodLookup.findVirtual( + tag.runtimeClass, + field.writeMethod.get, + MethodType.methodType(classOf[Unit], field.enc.clsTag.runtimeClass)) +(bean: Any, i: Int) => setter.invoke(bean, deserializer.get(i)) + } new StructFieldSerializer[Any](struct) { def value(i: Int): Any = { val instance = constructor.invoke() 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 16a7d7ff065..0b88d5a4130 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 @@ -390,7 +390,9 @@ object DeserializerBuildHelper { CreateExternalRow(convertedFields, enc.schema)) case JavaBeanEncoder(tag, fields) => - val setters = fields.map { f => + val setters = fields +.filter(_.writeMethod.isDefined) +.map { f => val newTypePath = walkedTypePath.recordField( f.enc.clsTag.runtimeClass.getName, f.name) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index 4f7cf8da787..f416d411322 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataset
[spark] branch branch-3.5 updated: [SPARK-45072][CONNECT] Fix outer scopes for ammonite 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 9389a2ccacce [SPARK-45072][CONNECT] Fix outer scopes for ammonite classes 9389a2ccacce is described below commit 9389a2ccacce61cbbbc9bbb1b19b2825d932ba11 Author: Herman van Hovell AuthorDate: Tue Sep 5 15:35:12 2023 +0200 [SPARK-45072][CONNECT] Fix outer scopes for ammonite classes ### What changes were proposed in this pull request? Ammonite places all user code inside Helper classes which are nested inside the class it creates for each command. This PR adds a custom code class wrapper for the Ammonite REPL. It makes sure the Helper classes generated by ammonite are always registered as an outer scope immediately. This way we can instantiate classes defined inside the Helper class, even when we execute Spark code as part of the Helper's constructor. ### Why are the changes needed? When you currently define a class and execute a Spark command using that class inside the same cell/line this will fail with an NullPointerException. The reason for that is that we cannot resolve the outer scope needed to instantiate the class. This PR fixes that issue. The following code will now execute successfully (include the curly braces): ```scala { case class Thing(val value: String) val r = (0 to 10).map( value => Thing(value.toString) ) spark.createDataFrame(r) } ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I added more tests to the `ReplE2ESuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42807 from hvanhovell/SPARK-45072. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 40943c2748fdd28d970d017cb8ee86c294ee62df) Signed-off-by: Herman van Hovell --- .../apache/spark/sql/application/ConnectRepl.scala | 29 +++-- .../spark/sql/application/ReplE2ESuite.scala | 48 ++ .../CheckConnectJvmClientCompatibility.scala | 6 +++ 3 files changed, 71 insertions(+), 12 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/application/ConnectRepl.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/application/ConnectRepl.scala index e6ada566398c..0360a4057886 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/application/ConnectRepl.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/application/ConnectRepl.scala @@ -22,7 +22,8 @@ import java.util.concurrent.Semaphore import scala.util.control.NonFatal import ammonite.compiler.CodeClassWrapper -import ammonite.util.Bind +import ammonite.compiler.iface.CodeWrapper +import ammonite.util.{Bind, Imports, Name, Util} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.SparkSession @@ -94,8 +95,8 @@ object ConnectRepl { val main = ammonite.Main( welcomeBanner = Option(splash), predefCode = predefCode, - replCodeWrapper = CodeClassWrapper, - scriptCodeWrapper = CodeClassWrapper, + replCodeWrapper = ExtendedCodeClassWrapper, + scriptCodeWrapper = ExtendedCodeClassWrapper, inputStream = inputStream, outputStream = outputStream, errorStream = errorStream) @@ -107,3 +108,25 @@ object ConnectRepl { } } } + +/** + * [[CodeWrapper]] that makes sure new Helper classes are always registered as an outer scope. + */ +@DeveloperApi +object ExtendedCodeClassWrapper extends CodeWrapper { + override def wrapperPath: Seq[Name] = CodeClassWrapper.wrapperPath + override def apply( + code: String, + source: Util.CodeSource, + imports: Imports, + printCode: String, + indexedWrapper: Name, + extraCode: String): (String, String, Int) = { +val (top, bottom, level) = + CodeClassWrapper(code, source, imports, printCode, indexedWrapper, extraCode) +// Make sure we register the Helper before anything else, so outer scopes work as expected. +val augmentedTop = top + + "\norg.apache.spark.sql.catalyst.encoders.OuterScopes.addOuterScope(this)\n" +(augmentedTop, bottom, level) + } +} diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala index 4106d298dbe2..5bb8cbf3543b 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala @@ -79,12 +79,10 @@ class
[spark] branch master updated: [SPARK-45072][CONNECT] Fix outer scopes for ammonite 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 40943c2748fd [SPARK-45072][CONNECT] Fix outer scopes for ammonite classes 40943c2748fd is described below commit 40943c2748fdd28d970d017cb8ee86c294ee62df Author: Herman van Hovell AuthorDate: Tue Sep 5 15:35:12 2023 +0200 [SPARK-45072][CONNECT] Fix outer scopes for ammonite classes ### What changes were proposed in this pull request? Ammonite places all user code inside Helper classes which are nested inside the class it creates for each command. This PR adds a custom code class wrapper for the Ammonite REPL. It makes sure the Helper classes generated by ammonite are always registered as an outer scope immediately. This way we can instantiate classes defined inside the Helper class, even when we execute Spark code as part of the Helper's constructor. ### Why are the changes needed? When you currently define a class and execute a Spark command using that class inside the same cell/line this will fail with an NullPointerException. The reason for that is that we cannot resolve the outer scope needed to instantiate the class. This PR fixes that issue. The following code will now execute successfully (include the curly braces): ```scala { case class Thing(val value: String) val r = (0 to 10).map( value => Thing(value.toString) ) spark.createDataFrame(r) } ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I added more tests to the `ReplE2ESuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42807 from hvanhovell/SPARK-45072. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../apache/spark/sql/application/ConnectRepl.scala | 29 +++-- .../spark/sql/application/ReplE2ESuite.scala | 48 ++ .../CheckConnectJvmClientCompatibility.scala | 6 +++ 3 files changed, 71 insertions(+), 12 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/application/ConnectRepl.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/application/ConnectRepl.scala index e6ada566398c..0360a4057886 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/application/ConnectRepl.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/application/ConnectRepl.scala @@ -22,7 +22,8 @@ import java.util.concurrent.Semaphore import scala.util.control.NonFatal import ammonite.compiler.CodeClassWrapper -import ammonite.util.Bind +import ammonite.compiler.iface.CodeWrapper +import ammonite.util.{Bind, Imports, Name, Util} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.SparkSession @@ -94,8 +95,8 @@ object ConnectRepl { val main = ammonite.Main( welcomeBanner = Option(splash), predefCode = predefCode, - replCodeWrapper = CodeClassWrapper, - scriptCodeWrapper = CodeClassWrapper, + replCodeWrapper = ExtendedCodeClassWrapper, + scriptCodeWrapper = ExtendedCodeClassWrapper, inputStream = inputStream, outputStream = outputStream, errorStream = errorStream) @@ -107,3 +108,25 @@ object ConnectRepl { } } } + +/** + * [[CodeWrapper]] that makes sure new Helper classes are always registered as an outer scope. + */ +@DeveloperApi +object ExtendedCodeClassWrapper extends CodeWrapper { + override def wrapperPath: Seq[Name] = CodeClassWrapper.wrapperPath + override def apply( + code: String, + source: Util.CodeSource, + imports: Imports, + printCode: String, + indexedWrapper: Name, + extraCode: String): (String, String, Int) = { +val (top, bottom, level) = + CodeClassWrapper(code, source, imports, printCode, indexedWrapper, extraCode) +// Make sure we register the Helper before anything else, so outer scopes work as expected. +val augmentedTop = top + + "\norg.apache.spark.sql.catalyst.encoders.OuterScopes.addOuterScope(this)\n" +(augmentedTop, bottom, level) + } +} diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala index 4106d298dbe2..5bb8cbf3543b 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala @@ -79,12 +79,10 @@ class ReplE2ESuite extends RemoteSparkSession with BeforeAndAfterEach { override def afterEach(): Unit = { semaphore
[spark] branch master updated: [SPARK-43923][CONNECT][FOLLOW-UP] Propagate extra tags to SparkListenerConnectOperationFinished
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 edd21069727 [SPARK-43923][CONNECT][FOLLOW-UP] Propagate extra tags to SparkListenerConnectOperationFinished edd21069727 is described below commit edd210697272c03e3d97a6443a65d0a130353c05 Author: Martin Grund AuthorDate: Wed Aug 30 17:35:09 2023 +0200 [SPARK-43923][CONNECT][FOLLOW-UP] Propagate extra tags to SparkListenerConnectOperationFinished ### What changes were proposed in this pull request? The `SparkListenerConnectOperationFinished` message supports passing extra tags, but the event method did not support them yet. This patch propagates the extra tags and adds a test for it. ### Why are the changes needed? Compatibility with the message interface. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT ### Was this patch authored or co-authored using generative AI tooling? No Closes #42732 from grundprinzip/SPARK-43923. Authored-by: Martin Grund Signed-off-by: Herman van Hovell --- .../spark/sql/connect/service/ExecuteEventsManager.scala| 7 +-- .../sql/connect/service/ExecuteEventsManagerSuite.scala | 13 + 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala index 23a67b7292b..9e8a945bcc3 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala @@ -205,7 +205,9 @@ case class ExecuteEventsManager(executeHolder: ExecuteHolder, clock: Clock) { * Number of rows that are returned to the user. None is expected when the operation does not * return any rows. */ - def postFinished(producedRowsCountOpt: Option[Long] = None): Unit = { + def postFinished( + producedRowsCountOpt: Option[Long] = None, + extraTags: Map[String, String] = Map.empty): Unit = { assertStatus( List(ExecuteStatus.Started, ExecuteStatus.ReadyForExecution), ExecuteStatus.Finished) @@ -217,7 +219,8 @@ case class ExecuteEventsManager(executeHolder: ExecuteHolder, clock: Clock) { jobTag, operationId, clock.getTimeMillis(), - producedRowCount)) + producedRowCount, + extraTags)) } /** diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ExecuteEventsManagerSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ExecuteEventsManagerSuite.scala index 12e67f2c59c..dbe8420eab0 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ExecuteEventsManagerSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ExecuteEventsManagerSuite.scala @@ -184,6 +184,19 @@ class ExecuteEventsManagerSuite Some(100))) } + test("SPARK-43923: post finished with extra tags") { +val events = setupEvents(ExecuteStatus.Started) +events.postFinished(Some(100), Map("someEvent" -> "true")) + verify(events.executeHolder.sessionHolder.session.sparkContext.listenerBus, times(1)) + .post( +SparkListenerConnectOperationFinished( + events.executeHolder.jobTag, + DEFAULT_QUERY_ID, + DEFAULT_CLOCK.getTimeMillis(), + Some(100), + Map("someEvent" -> "true"))) + } + test("SPARK-43923: post closed") { val events = setupEvents(ExecuteStatus.Finished) events.postClosed() - 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-44832][CONNECT] Make transitive dependencies work properly 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 fd07239505c [SPARK-44832][CONNECT] Make transitive dependencies work properly for Scala Client fd07239505c is described below commit fd07239505cff8baafa4f1684034278d41234de7 Author: Herman van Hovell AuthorDate: Mon Aug 28 19:53:40 2023 +0200 [SPARK-44832][CONNECT] Make transitive dependencies work properly for Scala Client ### What changes were proposed in this pull request? This PR cleans up the Maven build for the Spark Connect Client and Spark Connect Common. The most important change is that we move `sql-api` from a `provided` to `compile` dependency. The net effect of this is that when a user takes a dependency on the client, all of its required (transitive) dependencies are automatically added. Please note that this does not address concerns around creating an überjar and shading. That is for a different day :) ### Why are the changes needed? When you take a dependency on the connect scala client you need to manually add the `sql-api` module as a dependency. This is rather poor UX. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually running maven, checking dependency tree, ... Closes #42518 from hvanhovell/SPARK-44832. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 50d9a56f824ae51d10543f4573753ff60dc9053b) Signed-off-by: Herman van Hovell --- connector/connect/client/jvm/pom.xml | 48 -- .../CheckConnectJvmClientCompatibility.scala | 33 --- connector/connect/common/pom.xml | 6 --- dev/connect-jvm-client-mima-check | 2 +- 4 files changed, 28 insertions(+), 61 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index a7e5c5c2bab..67227ef38eb 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -39,55 +39,21 @@ org.apache.spark spark-connect-common_${scala.binary.version} ${project.version} - - - com.google.guava - guava - - org.apache.spark spark-sql-api_${scala.binary.version} ${project.version} - provided org.apache.spark spark-sketch_${scala.binary.version} ${project.version} - - com.google.protobuf - protobuf-java - compile - com.google.guava guava ${connect.guava.version} - compile - - - com.google.guava - failureaccess - ${guava.failureaccess.version} - compile - - - io.netty - netty-codec-http2 - ${netty.version} - - - io.netty - netty-handler-proxy - ${netty.version} - - - io.netty - netty-transport-native-unix-common - ${netty.version} com.lihaoyi @@ -95,19 +61,6 @@ ${ammonite.version} provided - - org.apache.spark - spark-connect-common_${scala.binary.version} - ${project.version} - test-jar - test - - - com.google.guava - guava - - - org.scalacheck scalacheck_${scala.binary.version} @@ -148,7 +101,6 @@ org.codehaus.mojo:* org.checkerframework:* org.apache.spark:spark-connect-common_${scala.binary.version} - org.apache.spark:spark-common-utils_${scala.binary.version} 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 1f599f2346e..72b0f02f378 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 @@ -24,6 +24,7 @@ import java.util.regex.Pattern import com.typesafe.tools.mima.core._ import com.typesafe.tools.mima.lib.MiMaLib +import org.apache.spark.SparkBuildInfo.spark_version import org.apache.spark.sql.test.IntegrationTestUtils._ /** @@ -46,18 +47,38 @@ object CheckConnectJvmClientCompatibility { sys.env("SPARK_HOME") } + private val sqlJar = { +val path = Paths.get( + sparkHome, + "sql", + "core", + "target", + "scala-"
[spark] branch master updated: [SPARK-44832][CONNECT] Make transitive dependencies work properly 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 50d9a56f824 [SPARK-44832][CONNECT] Make transitive dependencies work properly for Scala Client 50d9a56f824 is described below commit 50d9a56f824ae51d10543f4573753ff60dc9053b Author: Herman van Hovell AuthorDate: Mon Aug 28 19:53:40 2023 +0200 [SPARK-44832][CONNECT] Make transitive dependencies work properly for Scala Client ### What changes were proposed in this pull request? This PR cleans up the Maven build for the Spark Connect Client and Spark Connect Common. The most important change is that we move `sql-api` from a `provided` to `compile` dependency. The net effect of this is that when a user takes a dependency on the client, all of its required (transitive) dependencies are automatically added. Please note that this does not address concerns around creating an überjar and shading. That is for a different day :) ### Why are the changes needed? When you take a dependency on the connect scala client you need to manually add the `sql-api` module as a dependency. This is rather poor UX. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually running maven, checking dependency tree, ... Closes #42518 from hvanhovell/SPARK-44832. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- connector/connect/client/jvm/pom.xml | 48 -- .../CheckConnectJvmClientCompatibility.scala | 33 --- connector/connect/common/pom.xml | 6 --- dev/connect-jvm-client-mima-check | 2 +- 4 files changed, 28 insertions(+), 61 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index d4e9b147e02..8cb6758ec9f 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -39,55 +39,21 @@ org.apache.spark spark-connect-common_${scala.binary.version} ${project.version} - - - com.google.guava - guava - - org.apache.spark spark-sql-api_${scala.binary.version} ${project.version} - provided org.apache.spark spark-sketch_${scala.binary.version} ${project.version} - - com.google.protobuf - protobuf-java - compile - com.google.guava guava ${connect.guava.version} - compile - - - com.google.guava - failureaccess - ${guava.failureaccess.version} - compile - - - io.netty - netty-codec-http2 - ${netty.version} - - - io.netty - netty-handler-proxy - ${netty.version} - - - io.netty - netty-transport-native-unix-common - ${netty.version} com.lihaoyi @@ -95,19 +61,6 @@ ${ammonite.version} provided - - org.apache.spark - spark-connect-common_${scala.binary.version} - ${project.version} - test-jar - test - - - com.google.guava - guava - - - org.scalacheck scalacheck_${scala.binary.version} @@ -148,7 +101,6 @@ org.codehaus.mojo:* org.checkerframework:* org.apache.spark:spark-connect-common_${scala.binary.version} - org.apache.spark:spark-common-utils_${scala.binary.version} 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 1100babde79..1e536cd37fe 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 @@ -24,6 +24,7 @@ import java.util.regex.Pattern import com.typesafe.tools.mima.core._ import com.typesafe.tools.mima.lib.MiMaLib +import org.apache.spark.SparkBuildInfo.spark_version import org.apache.spark.sql.test.IntegrationTestUtils._ /** @@ -46,18 +47,38 @@ object CheckConnectJvmClientCompatibility { sys.env("SPARK_HOME") } + private val sqlJar = { +val path = Paths.get( + sparkHome, + "sql", + "core", + "target", + "scala-" + scalaVersion, + "spark-sql_" + scalaVersion + "-" + spark_version + ".jar&
[spark] branch branch-3.5 updated: [SPARK-44867][CONNECT][DOCS] Refactor Spark Connect Docs to incorporate Scala setup
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 12964c26a45 [SPARK-44867][CONNECT][DOCS] Refactor Spark Connect Docs to incorporate Scala setup 12964c26a45 is described below commit 12964c26a4511bc21005885e21ef572a69dde7c2 Author: vicennial AuthorDate: Mon Aug 28 16:38:25 2023 +0200 [SPARK-44867][CONNECT][DOCS] Refactor Spark Connect Docs to incorporate Scala setup ### What changes were proposed in this pull request? This PR refactors the Spark Connect overview docs to include an Interactive (shell/REPL) section and a Standalone application section as well as incorporates new Scala documentation into each of these sections. ### Why are the changes needed? Currently, there isn't much Scala-relevant documentation available to set up the Scala shell/project/application. ### Does this PR introduce _any_ user-facing change? Yes, the documentation for the Spark Connect [overview](https://spark.apache.org/docs/latest/spark-connect-overview.html) page is updated. ### How was this patch tested? Manually generating the docs locally. Closes #42556 from vicennial/sparkConnectDocs. Authored-by: vicennial Signed-off-by: Herman van Hovell (cherry picked from commit d95e8f3c65e5ae0bf39c0ccc477b7b0910513066) Signed-off-by: Herman van Hovell --- docs/spark-connect-overview.md | 204 ++--- 1 file changed, 170 insertions(+), 34 deletions(-) diff --git a/docs/spark-connect-overview.md b/docs/spark-connect-overview.md index 1e1464cfba0..0673763f03b 100644 --- a/docs/spark-connect-overview.md +++ b/docs/spark-connect-overview.md @@ -113,14 +113,15 @@ Now Spark server is running and ready to accept Spark Connect sessions from clie applications. In the next section we will walk through how to use Spark Connect when writing client applications. -## Use Spark Connect in client applications +## Use Spark Connect for interactive analysis + + When creating a Spark session, you can specify that you want to use Spark Connect and there are a few ways to do that outlined as follows. If you do not use one of the mechanisms outlined here, your Spark session will -work just like before, without leveraging Spark Connect, and your application code -will run on the Spark driver node. +work just like before, without leveraging Spark Connect. ### Set SPARK_REMOTE environment variable @@ -138,9 +139,6 @@ export SPARK_REMOTE="sc://localhost" And start the Spark shell as usual: - - - {% highlight bash %} ./bin/pyspark {% endhighlight %} @@ -150,25 +148,6 @@ The PySpark shell is now connected to Spark using Spark Connect as indicated in {% highlight python %} Client connected to the Spark Connect server at localhost {% endhighlight %} - - - - -And if you write your own program, create a Spark session as shown in this example: - - - - -{% highlight python %} -from pyspark.sql import SparkSession -spark = SparkSession.builder.getOrCreate() -{% endhighlight %} - - - - -This will create a Spark Connect session from your application by reading the -`SPARK_REMOTE` environment variable we set previously. ### Specify Spark Connect when creating Spark session @@ -178,9 +157,6 @@ create a Spark session. For example, you can launch the PySpark shell with Spark Connect as illustrated here. - - - To launch the PySpark shell with Spark Connect, simply include the `remote` parameter and specify the location of your Spark server. We are using `localhost` in this example to connect to the local Spark server we started previously: @@ -219,29 +195,175 @@ Now you can run PySpark code in the shell to see Spark Connect in action: | 2|Maria| +---+-+ {% endhighlight %} + + +For the Scala shell, we use an Ammonite-based REPL that is currently not included in the Apache Spark package. + +To set up the new Scala shell, first download and install [Coursier CLI](https://get-coursier.io/docs/cli-installation). +Then, install the REPL using the following command in a terminal window: +{% highlight bash %} +cs install –-contrib spark-connect-repl +{% endhighlight %} + +And now you can start the Ammonite-based Scala REPL/shell to connect to your Spark server like this: + +{% highlight bash %} +spark-connect-repl +{% endhighlight %} + +A greeting message will appear when the REPL successfully initializes: +{% highlight bash %} +Spark session available
[spark] branch master updated: [SPARK-44867][CONNECT][DOCS] Refactor Spark Connect Docs to incorporate Scala setup
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 d95e8f3c65e [SPARK-44867][CONNECT][DOCS] Refactor Spark Connect Docs to incorporate Scala setup d95e8f3c65e is described below commit d95e8f3c65e5ae0bf39c0ccc477b7b0910513066 Author: vicennial AuthorDate: Mon Aug 28 16:38:25 2023 +0200 [SPARK-44867][CONNECT][DOCS] Refactor Spark Connect Docs to incorporate Scala setup ### What changes were proposed in this pull request? This PR refactors the Spark Connect overview docs to include an Interactive (shell/REPL) section and a Standalone application section as well as incorporates new Scala documentation into each of these sections. ### Why are the changes needed? Currently, there isn't much Scala-relevant documentation available to set up the Scala shell/project/application. ### Does this PR introduce _any_ user-facing change? Yes, the documentation for the Spark Connect [overview](https://spark.apache.org/docs/latest/spark-connect-overview.html) page is updated. ### How was this patch tested? Manually generating the docs locally. Closes #42556 from vicennial/sparkConnectDocs. Authored-by: vicennial Signed-off-by: Herman van Hovell --- docs/spark-connect-overview.md | 204 ++--- 1 file changed, 170 insertions(+), 34 deletions(-) diff --git a/docs/spark-connect-overview.md b/docs/spark-connect-overview.md index 1e1464cfba0..0673763f03b 100644 --- a/docs/spark-connect-overview.md +++ b/docs/spark-connect-overview.md @@ -113,14 +113,15 @@ Now Spark server is running and ready to accept Spark Connect sessions from clie applications. In the next section we will walk through how to use Spark Connect when writing client applications. -## Use Spark Connect in client applications +## Use Spark Connect for interactive analysis + + When creating a Spark session, you can specify that you want to use Spark Connect and there are a few ways to do that outlined as follows. If you do not use one of the mechanisms outlined here, your Spark session will -work just like before, without leveraging Spark Connect, and your application code -will run on the Spark driver node. +work just like before, without leveraging Spark Connect. ### Set SPARK_REMOTE environment variable @@ -138,9 +139,6 @@ export SPARK_REMOTE="sc://localhost" And start the Spark shell as usual: - - - {% highlight bash %} ./bin/pyspark {% endhighlight %} @@ -150,25 +148,6 @@ The PySpark shell is now connected to Spark using Spark Connect as indicated in {% highlight python %} Client connected to the Spark Connect server at localhost {% endhighlight %} - - - - -And if you write your own program, create a Spark session as shown in this example: - - - - -{% highlight python %} -from pyspark.sql import SparkSession -spark = SparkSession.builder.getOrCreate() -{% endhighlight %} - - - - -This will create a Spark Connect session from your application by reading the -`SPARK_REMOTE` environment variable we set previously. ### Specify Spark Connect when creating Spark session @@ -178,9 +157,6 @@ create a Spark session. For example, you can launch the PySpark shell with Spark Connect as illustrated here. - - - To launch the PySpark shell with Spark Connect, simply include the `remote` parameter and specify the location of your Spark server. We are using `localhost` in this example to connect to the local Spark server we started previously: @@ -219,29 +195,175 @@ Now you can run PySpark code in the shell to see Spark Connect in action: | 2|Maria| +---+-+ {% endhighlight %} + + +For the Scala shell, we use an Ammonite-based REPL that is currently not included in the Apache Spark package. + +To set up the new Scala shell, first download and install [Coursier CLI](https://get-coursier.io/docs/cli-installation). +Then, install the REPL using the following command in a terminal window: +{% highlight bash %} +cs install –-contrib spark-connect-repl +{% endhighlight %} + +And now you can start the Ammonite-based Scala REPL/shell to connect to your Spark server like this: + +{% highlight bash %} +spark-connect-repl +{% endhighlight %} + +A greeting message will appear when the REPL successfully initializes: +{% highlight bash %} +Spark session available as 'spark'. + _ __ ____ + / ___/ __/ /__ / /___ ___ _/ /_ + \__ \/ __ \/ __ `/ ___/ //_/ / / / __ \/ __ \/ __ \/ _ \/ ___/ __/ + ___/ / /_/ / /_/ / / / ,</ /___/ /_/ / / / / / / / __/ /__/ /_ +// .___/\__,_/_/ /_/|_| \/\/_/ /_/_/ /_/\___/\___/\__/ +/_/ +{% endhighlight %} + +By defa
[spark] branch branch-3.5 updated: [SPARK-44974][CONNECT] Null out SparkSession/Dataset/KeyValueGroupedDatset on serialization
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 c230a5011a6 [SPARK-44974][CONNECT] Null out SparkSession/Dataset/KeyValueGroupedDatset on serialization c230a5011a6 is described below commit c230a5011a6d45c0f393833995b052930f11c324 Author: Herman van Hovell AuthorDate: Mon Aug 28 15:05:18 2023 +0200 [SPARK-44974][CONNECT] Null out SparkSession/Dataset/KeyValueGroupedDatset on serialization ### What changes were proposed in this pull request? This PR changes the serialization for connect `SparkSession`, `Dataset`, and `KeyValueGroupedDataset`. While these were marked as serializable they were not, because they refer to bits and pieces that are not serializable. Even if we were to fix this, then we still have a class clash problem with server side classes that have the same name, but have different structure. the latter can be fixed with serialization proxies, but I am going to hold that until someone actually needs/wants this. After this PR these classes are serialized as null. This is a somewhat suboptimal solution compared to throwing exceptions on serialization, however this is more compatible compared to the old situation, and makes accidental capture of these classes less of an issue for UDFs. ### Why are the changes needed? More compatible with the old situation. Improved UX when working with UDFs. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests to `ClientDatasetSuite`, `KeyValueGroupedDatasetE2ETestSuite`, `SparkSessionSuite`, and `UserDefinedFunctionE2ETestSuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42688 from hvanhovell/SPARK-44974. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit f0b04286022e0774d78b9adcf4aeabc181a3ec89) Signed-off-by: Herman van Hovell --- .../jvm/src/main/scala/org/apache/spark/sql/Dataset.scala | 6 ++ .../org/apache/spark/sql/KeyValueGroupedDataset.scala | 6 ++ .../main/scala/org/apache/spark/sql/SparkSession.scala| 6 ++ .../scala/org/apache/spark/sql/ClientDatasetSuite.scala | 8 .../spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala| 7 +++ .../scala/org/apache/spark/sql/SparkSessionSuite.scala| 7 +++ .../spark/sql/UserDefinedFunctionE2ETestSuite.scala | 15 +++ 7 files changed, 55 insertions(+) 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 cb7d2c84df5..bdaa4e28ba8 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 @@ -3336,4 +3336,10 @@ class Dataset[T] private[sql] ( result.close() } } + + /** + * We cannot deserialize a connect [[Dataset]] because of a class clash on the server side. We + * null out the instance for now. + */ + private def writeReplace(): Any = null } 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 202891c66d7..88c8b6a4f8b 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 @@ -979,6 +979,12 @@ private class KeyValueGroupedDatasetImpl[K, V, IK, IV]( outputEncoder = outputEncoder) udf.apply(inputEncoders.map(_ => col("*")): _*).expr.getCommonInlineUserDefinedFunction } + + /** + * We cannot deserialize a connect [[KeyValueGroupedDataset]] because of a class clash on the + * server side. We null out the instance for now. + */ + private def writeReplace(): Any = null } private object KeyValueGroupedDatasetImpl { 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 e902e04e246..7882ea64013 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 @@ -714,6 +714,12 @@ class SparkSession private[sql] ( def clearTags(): Unit = { client.clearTags() } + + /** + * We cannot deserialize a connect [[SparkSession]] because of a class clash on the server side. + * We null out the ins
[spark] branch master updated: [SPARK-44974][CONNECT] Null out SparkSession/Dataset/KeyValueGroupedDatset on 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 f0b04286022 [SPARK-44974][CONNECT] Null out SparkSession/Dataset/KeyValueGroupedDatset on serialization f0b04286022 is described below commit f0b04286022e0774d78b9adcf4aeabc181a3ec89 Author: Herman van Hovell AuthorDate: Mon Aug 28 15:05:18 2023 +0200 [SPARK-44974][CONNECT] Null out SparkSession/Dataset/KeyValueGroupedDatset on serialization ### What changes were proposed in this pull request? This PR changes the serialization for connect `SparkSession`, `Dataset`, and `KeyValueGroupedDataset`. While these were marked as serializable they were not, because they refer to bits and pieces that are not serializable. Even if we were to fix this, then we still have a class clash problem with server side classes that have the same name, but have different structure. the latter can be fixed with serialization proxies, but I am going to hold that until someone actually needs/wants this. After this PR these classes are serialized as null. This is a somewhat suboptimal solution compared to throwing exceptions on serialization, however this is more compatible compared to the old situation, and makes accidental capture of these classes less of an issue for UDFs. ### Why are the changes needed? More compatible with the old situation. Improved UX when working with UDFs. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests to `ClientDatasetSuite`, `KeyValueGroupedDatasetE2ETestSuite`, `SparkSessionSuite`, and `UserDefinedFunctionE2ETestSuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42688 from hvanhovell/SPARK-44974. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../jvm/src/main/scala/org/apache/spark/sql/Dataset.scala | 6 ++ .../org/apache/spark/sql/KeyValueGroupedDataset.scala | 6 ++ .../main/scala/org/apache/spark/sql/SparkSession.scala| 6 ++ .../scala/org/apache/spark/sql/ClientDatasetSuite.scala | 8 .../spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala| 7 +++ .../scala/org/apache/spark/sql/SparkSessionSuite.scala| 7 +++ .../spark/sql/UserDefinedFunctionE2ETestSuite.scala | 15 +++ 7 files changed, 55 insertions(+) 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 3c89e649020..1d83f196b53 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 @@ -3352,4 +3352,10 @@ class Dataset[T] private[sql] ( result.close() } } + + /** + * We cannot deserialize a connect [[Dataset]] because of a class clash on the server side. We + * null out the instance for now. + */ + private def writeReplace(): Any = null } 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 202891c66d7..88c8b6a4f8b 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 @@ -979,6 +979,12 @@ private class KeyValueGroupedDatasetImpl[K, V, IK, IV]( outputEncoder = outputEncoder) udf.apply(inputEncoders.map(_ => col("*")): _*).expr.getCommonInlineUserDefinedFunction } + + /** + * We cannot deserialize a connect [[KeyValueGroupedDataset]] because of a class clash on the + * server side. We null out the instance for now. + */ + private def writeReplace(): Any = null } private object KeyValueGroupedDatasetImpl { 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 e902e04e246..7882ea64013 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 @@ -714,6 +714,12 @@ class SparkSession private[sql] ( def clearTags(): Unit = { client.clearTags() } + + /** + * We cannot deserialize a connect [[SparkSession]] because of a class clash on the server side. + * We null out the instance for now. + */ + private def writeReplace(): Any = null } // The minimal builder needed to create a spark session. diff --git
[spark] branch branch-3.5 updated: [SPARK-44816][CONNECT] Improve error message when UDF class is not found
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 2d7e05ddd98 [SPARK-44816][CONNECT] Improve error message when UDF class is not found 2d7e05ddd98 is described below commit 2d7e05ddd980ad7a934e5c8fde77335ef6591c06 Author: Niranjan Jayakar AuthorDate: Wed Aug 23 18:42:04 2023 +0200 [SPARK-44816][CONNECT] Improve error message when UDF class is not found ### What changes were proposed in this pull request? Improve the error messaging on the connect client when using a UDF whose corresponding class has not been sync'ed with the spark connect service. Prior to this change, the client receives a cryptic error: ``` Exception in thread "main" org.apache.spark.SparkException: Main$ ``` With this change, the message is improved to be: ``` Exception in thread "main" org.apache.spark.SparkException: Failed to load class: Main$. Make sure the artifact where the class is defined is installed by calling session.addArtifact. ``` ### Why are the changes needed? This change makes it clear to the user on what the error is. ### Does this PR introduce _any_ user-facing change? Yes. The error message is improved. See details above. ### How was this patch tested? Manually by running a connect server and client. Closes #42500 from nija-at/improve-error. Authored-by: Niranjan Jayakar Signed-off-by: Herman van Hovell (cherry picked from commit 2d0a0a00cb5dde6bcb8e561278357b6bb8b76dcc) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala | 5 + 1 file changed, 5 insertions(+) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index e81e9bb51cb..46c465e4deb 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -1527,6 +1527,11 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { s"Failed to load class correctly due to $nsm. " + "Make sure the artifact where the class is defined is installed by calling" + " session.addArtifact.") + case cnf: ClassNotFoundException => +throw new ClassNotFoundException( + s"Failed to load class: ${cnf.getMessage}. " + +"Make sure the artifact where the class is defined is installed by calling" + +" session.addArtifact.") case _ => throw t } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44816][CONNECT] Improve error message when UDF class is not found
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 2d0a0a00cb5 [SPARK-44816][CONNECT] Improve error message when UDF class is not found 2d0a0a00cb5 is described below commit 2d0a0a00cb5dde6bcb8e561278357b6bb8b76dcc Author: Niranjan Jayakar AuthorDate: Wed Aug 23 18:42:04 2023 +0200 [SPARK-44816][CONNECT] Improve error message when UDF class is not found ### What changes were proposed in this pull request? Improve the error messaging on the connect client when using a UDF whose corresponding class has not been sync'ed with the spark connect service. Prior to this change, the client receives a cryptic error: ``` Exception in thread "main" org.apache.spark.SparkException: Main$ ``` With this change, the message is improved to be: ``` Exception in thread "main" org.apache.spark.SparkException: Failed to load class: Main$. Make sure the artifact where the class is defined is installed by calling session.addArtifact. ``` ### Why are the changes needed? This change makes it clear to the user on what the error is. ### Does this PR introduce _any_ user-facing change? Yes. The error message is improved. See details above. ### How was this patch tested? Manually by running a connect server and client. Closes #42500 from nija-at/improve-error. Authored-by: Niranjan Jayakar Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala | 5 + 1 file changed, 5 insertions(+) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 70e6e926613..5b5018a1668 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -1523,6 +1523,11 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { s"Failed to load class correctly due to $nsm. " + "Make sure the artifact where the class is defined is installed by calling" + " session.addArtifact.") + case cnf: ClassNotFoundException => +throw new ClassNotFoundException( + s"Failed to load class: ${cnf.getMessage}. " + +"Make sure the artifact where the class is defined is installed by calling" + +" session.addArtifact.") case _ => throw t } } - 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-44861][CONNECT] jsonignore SparkListenerConnectOperationStarted.planRequest
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 a7941f15a0c [SPARK-44861][CONNECT] jsonignore SparkListenerConnectOperationStarted.planRequest a7941f15a0c is described below commit a7941f15a0c3034888b1adbd5affce2a9e12788e Author: jdesjean AuthorDate: Wed Aug 23 18:39:49 2023 +0200 [SPARK-44861][CONNECT] jsonignore SparkListenerConnectOperationStarted.planRequest ### What changes were proposed in this pull request? Add `JsonIgnore` to `SparkListenerConnectOperationStarted.planRequest` ### Why are the changes needed? `SparkListenerConnectOperationStarted` was added as part of [SPARK-43923](https://issues.apache.org/jira/browse/SPARK-43923). `SparkListenerConnectOperationStarted.planRequest` cannot be serialized & deserialized from json as it has recursive objects which causes failures when attempting these operations. ``` com.fasterxml.jackson.databind.exc.InvalidDefinitionException: Direct self-reference leading to cycle (through reference chain: org.apache.spark.sql.connect.service.SparkListenerConnectOperationStarted["planRequest"]->org.apache.spark.connect.proto.ExecutePlanRequest["unknownFields"]->grpc_shaded.com.google.protobuf.UnknownFieldSet["defaultInstanceForType"]) at com.fasterxml.jackson.databind.exc.InvalidDefinitionException.from(InvalidDefinitionException.java:77) at com.fasterxml.jackson.databind.SerializerProvider.reportBadDefinition(SerializerProvider.java:1308) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit Closes #42550 from jdesjean/SPARK-44861. Authored-by: jdesjean Signed-off-by: Herman van Hovell (cherry picked from commit dd6cda5b614b4ede418afb4c5b1fdeea9613d32c) Signed-off-by: Herman van Hovell --- .../sql/connect/service/ExecuteEventsManager.scala | 37 --- .../service/ExecuteEventsManagerSuite.scala| 114 ++--- .../ui/SparkConnectServerListenerSuite.scala | 3 - .../connect/ui/SparkConnectServerPageSuite.scala | 1 - 4 files changed, 95 insertions(+), 60 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala index 5b9267a9679..23a67b7292b 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala @@ -119,19 +119,19 @@ case class ExecuteEventsManager(executeHolder: ExecuteHolder, clock: Clock) { s"${request.getPlan.getOpTypeCase} not supported.") } -listenerBus.post( - SparkListenerConnectOperationStarted( -jobTag, -operationId, -clock.getTimeMillis(), -sessionId, -request.getUserContext.getUserId, -request.getUserContext.getUserName, -Utils.redact( - sessionHolder.session.sessionState.conf.stringRedactionPattern, - ProtoUtils.abbreviate(plan, ExecuteEventsManager.MAX_STATEMENT_TEXT_SIZE).toString), -Some(request), -sparkSessionTags)) +val event = SparkListenerConnectOperationStarted( + jobTag, + operationId, + clock.getTimeMillis(), + sessionId, + request.getUserContext.getUserId, + request.getUserContext.getUserName, + Utils.redact( +sessionHolder.session.sessionState.conf.stringRedactionPattern, +ProtoUtils.abbreviate(plan, ExecuteEventsManager.MAX_STATEMENT_TEXT_SIZE).toString), + sparkSessionTags) +event.planRequest = Some(request) +listenerBus.post(event) } /** @@ -290,8 +290,6 @@ case class ExecuteEventsManager(executeHolder: ExecuteHolder, clock: Clock) { * Opaque userName set in the Connect request. * @param statementText: * The connect request plan converted to text. - * @param planRequest: - * The Connect request. None if the operation is not of type @link proto.ExecutePlanRequest * @param sparkSessionTags: * Extra tags set by the user (via SparkSession.addTag). * @param extraTags: @@ -305,10 +303,15 @@ case class SparkListenerConnectOperationStarted( userId: String, userName: String, statementText: String, -planRequest: Option[proto.ExecutePlanRequest], sparkSessionTags: Set[String], extraTags: Map[String, String] = Map.empty) -extends SparkListenerEvent +extends SparkListenerEvent { + + /** + * The Connect request. None if the operation is not of type @link proto.Execut
[spark] branch master updated: [SPARK-44861][CONNECT] jsonignore SparkListenerConnectOperationStarted.planRequest
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 dd6cda5b614 [SPARK-44861][CONNECT] jsonignore SparkListenerConnectOperationStarted.planRequest dd6cda5b614 is described below commit dd6cda5b614b4ede418afb4c5b1fdeea9613d32c Author: jdesjean AuthorDate: Wed Aug 23 18:39:49 2023 +0200 [SPARK-44861][CONNECT] jsonignore SparkListenerConnectOperationStarted.planRequest ### What changes were proposed in this pull request? Add `JsonIgnore` to `SparkListenerConnectOperationStarted.planRequest` ### Why are the changes needed? `SparkListenerConnectOperationStarted` was added as part of [SPARK-43923](https://issues.apache.org/jira/browse/SPARK-43923). `SparkListenerConnectOperationStarted.planRequest` cannot be serialized & deserialized from json as it has recursive objects which causes failures when attempting these operations. ``` com.fasterxml.jackson.databind.exc.InvalidDefinitionException: Direct self-reference leading to cycle (through reference chain: org.apache.spark.sql.connect.service.SparkListenerConnectOperationStarted["planRequest"]->org.apache.spark.connect.proto.ExecutePlanRequest["unknownFields"]->grpc_shaded.com.google.protobuf.UnknownFieldSet["defaultInstanceForType"]) at com.fasterxml.jackson.databind.exc.InvalidDefinitionException.from(InvalidDefinitionException.java:77) at com.fasterxml.jackson.databind.SerializerProvider.reportBadDefinition(SerializerProvider.java:1308) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit Closes #42550 from jdesjean/SPARK-44861. Authored-by: jdesjean Signed-off-by: Herman van Hovell --- .../sql/connect/service/ExecuteEventsManager.scala | 37 --- .../service/ExecuteEventsManagerSuite.scala| 114 ++--- .../ui/SparkConnectServerListenerSuite.scala | 3 - .../connect/ui/SparkConnectServerPageSuite.scala | 1 - 4 files changed, 95 insertions(+), 60 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala index 5b9267a9679..23a67b7292b 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala @@ -119,19 +119,19 @@ case class ExecuteEventsManager(executeHolder: ExecuteHolder, clock: Clock) { s"${request.getPlan.getOpTypeCase} not supported.") } -listenerBus.post( - SparkListenerConnectOperationStarted( -jobTag, -operationId, -clock.getTimeMillis(), -sessionId, -request.getUserContext.getUserId, -request.getUserContext.getUserName, -Utils.redact( - sessionHolder.session.sessionState.conf.stringRedactionPattern, - ProtoUtils.abbreviate(plan, ExecuteEventsManager.MAX_STATEMENT_TEXT_SIZE).toString), -Some(request), -sparkSessionTags)) +val event = SparkListenerConnectOperationStarted( + jobTag, + operationId, + clock.getTimeMillis(), + sessionId, + request.getUserContext.getUserId, + request.getUserContext.getUserName, + Utils.redact( +sessionHolder.session.sessionState.conf.stringRedactionPattern, +ProtoUtils.abbreviate(plan, ExecuteEventsManager.MAX_STATEMENT_TEXT_SIZE).toString), + sparkSessionTags) +event.planRequest = Some(request) +listenerBus.post(event) } /** @@ -290,8 +290,6 @@ case class ExecuteEventsManager(executeHolder: ExecuteHolder, clock: Clock) { * Opaque userName set in the Connect request. * @param statementText: * The connect request plan converted to text. - * @param planRequest: - * The Connect request. None if the operation is not of type @link proto.ExecutePlanRequest * @param sparkSessionTags: * Extra tags set by the user (via SparkSession.addTag). * @param extraTags: @@ -305,10 +303,15 @@ case class SparkListenerConnectOperationStarted( userId: String, userName: String, statementText: String, -planRequest: Option[proto.ExecutePlanRequest], sparkSessionTags: Set[String], extraTags: Map[String, String] = Map.empty) -extends SparkListenerEvent +extends SparkListenerEvent { + + /** + * The Connect request. None if the operation is not of type @link proto.ExecutePlanRequest + */ + @JsonIgnore var planRequest: Option[proto.ExecutePlanRequest] = None +} /** * The event is sen
[spark] branch master updated (be04ac1ace9 -> 6771d9d757f)
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 be04ac1ace9 [SPARK-44834][PYTHON][SQL][TESTS] Add SQL query tests for Python UDTFs add 6771d9d757f [SPARK-44433][3.5X] Terminate foreach batch runner when streaming query terminates No new revisions were added by this update. Summary of changes: .../sql/connect/planner/SparkConnectPlanner.scala | 37 +-- .../planner/StreamingForeachBatchHelper.scala | 109 ++--- .../spark/sql/connect/service/SessionHolder.scala | 9 +- .../service/SparkConnectStreamingQueryCache.scala | 21 ++-- .../planner/StreamingForeachBatchHelperSuite.scala | 80 +++ .../spark/api/python/PythonWorkerFactory.scala | 2 +- .../spark/api/python/StreamingPythonRunner.scala | 9 +- 7 files changed, 231 insertions(+), 36 deletions(-) create mode 100644 connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelperSuite.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-44849] Expose SparkConnectExecutionManager.listActiveExecutions
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 893940db795 [SPARK-44849] Expose SparkConnectExecutionManager.listActiveExecutions 893940db795 is described below commit 893940db79500d7faf3263cd75b96a814aa0f279 Author: Juliusz Sompolski AuthorDate: Thu Aug 17 17:49:28 2023 +0200 [SPARK-44849] Expose SparkConnectExecutionManager.listActiveExecutions ### What changes were proposed in this pull request? Make list of active executions accessible via SparkConnectService.listActiveExecutions ### Why are the changes needed? Some internal components outside `connect` would like to have access to that. Add a method to expose it. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? CI Closes #42535 from juliuszsompolski/SPARK-44849. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell (cherry picked from commit 047b2247879cc3f5fd9b78366a73edbf62994811) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/connect/service/SparkConnectService.scala | 6 ++ 1 file changed, 6 insertions(+) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index 15a7782c367..e8af2acfd2e 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -347,6 +347,12 @@ object SparkConnectService extends Logging { userSessionMapping.get((userId, sessionId), default) } + /** + * If there are no executions, return Left with System.currentTimeMillis of last active + * execution. Otherwise return Right with list of ExecuteInfo of all executions. + */ + def listActiveExecutions: Either[Long, Seq[ExecuteInfo]] = executionManager.listActiveExecutions + /** * Used for testing */ - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44849] Expose SparkConnectExecutionManager.listActiveExecutions
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 047b2247879 [SPARK-44849] Expose SparkConnectExecutionManager.listActiveExecutions 047b2247879 is described below commit 047b2247879cc3f5fd9b78366a73edbf62994811 Author: Juliusz Sompolski AuthorDate: Thu Aug 17 17:49:28 2023 +0200 [SPARK-44849] Expose SparkConnectExecutionManager.listActiveExecutions ### What changes were proposed in this pull request? Make list of active executions accessible via SparkConnectService.listActiveExecutions ### Why are the changes needed? Some internal components outside `connect` would like to have access to that. Add a method to expose it. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? CI Closes #42535 from juliuszsompolski/SPARK-44849. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/connect/service/SparkConnectService.scala | 6 ++ 1 file changed, 6 insertions(+) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index fe773e4b704..269e47609db 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -348,6 +348,12 @@ object SparkConnectService extends Logging { userSessionMapping.get((userId, sessionId), default) } + /** + * If there are no executions, return Left with System.currentTimeMillis of last active + * execution. Otherwise return Right with list of ExecuteInfo of all executions. + */ + def listActiveExecutions: Either[Long, Seq[ExecuteInfo]] = executionManager.listActiveExecutions + /** * Used for testing */ - 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-44806][CONNECT] Move internal client spark-connect-common to be able to test real in-process server with a real RPC 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 15bb95564b6 [SPARK-44806][CONNECT] Move internal client spark-connect-common to be able to test real in-process server with a real RPC client 15bb95564b6 is described below commit 15bb95564b6f2a3e76996062155ea029438ab4bb Author: Juliusz Sompolski AuthorDate: Thu Aug 17 17:05:58 2023 +0200 [SPARK-44806][CONNECT] Move internal client spark-connect-common to be able to test real in-process server with a real RPC client ### What changes were proposed in this pull request? Currently, Spark Connect has the following types of integration tests: * Client unit tests with a mock in process server (DummySparkConnectService with GRPC InProcessServerBuilder) * Client unit tests with a mock RPC server (DummySparkConnectService with GRPC NettyServerBuilder) * Server unit tests with in-process server and using various mocks * E2E tests of real client with a server started in another process (using RemoteSparkSession) What is lacking are E2E tests with an in-process Server (so that server state can be inspected asserted), and a real RPC client. This is impossible, because classes from `spark-connect-client-jvm` module include the client API which duplicates Spark SQL APIs of SparkSession, Dataset etc. When trying to pull a real client into the server module for testing, these classes clash. Move the `org.apache.spark.sql.connect.client` code into `spark-connect-common` module, so that the internal SparkConnectClient code is separated from the client public API, and can be pulled into testing of the server. The only class that we keep in `spark-connect-client-jvm` is `AmmoniteClassFinder`, to avoid pulling in ammonite dependency into common. Tried alternative approach in https://github.com/apache/spark/pull/42465. That doesn't work, because it also reorders the maven build in a way so that client is build before server, but client actually requires server to be build first to use tests with `RemoteSparkSession` Tried alternative approach to depend on a shaded/relocated version of client in https://github.com/apache/spark/pull/42461, but that's just not possible to do neither in maven nor sbt. Tried alternative approach to create client-jvm-internal module in https://github.com/apache/spark/pull/42441, moving things to connect-common was preferred to introducing new module by reviewers. Moved it together with tests in https://github.com/apache/spark/pull/42501, but moving tests isn't really needed. ### Why are the changes needed? For being able to use the internal client for testing of in-process server with an in-process client, but communicating over real RPC. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? All modules test and build. Closes #42523 from juliuszsompolski/sc-client-common-mainonly. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell (cherry picked from commit 72a466835a4490257eec0c9af2bbc9291c46de1e) Signed-off-by: Herman van Hovell --- ...ClassFinder.scala => AmmoniteClassFinder.scala} | 41 ++ connector/connect/common/pom.xml | 12 +++ .../client/arrow/ScalaCollectionUtils.scala| 0 .../client/arrow/ScalaCollectionUtils.scala| 0 .../spark/sql/connect/client/ArtifactManager.scala | 0 .../spark/sql/connect/client/ClassFinder.scala | 27 +- .../sql/connect/client/CloseableIterator.scala | 0 .../client/CustomSparkConnectBlockingStub.scala| 0 .../connect/client/CustomSparkConnectStub.scala| 0 .../ExecutePlanResponseReattachableIterator.scala | 0 .../connect/client/GrpcExceptionConverter.scala| 2 +- .../sql/connect/client/GrpcRetryHandler.scala | 0 .../sql/connect/client/SparkConnectClient.scala| 0 .../connect/client/SparkConnectClientParser.scala | 0 .../spark/sql/connect/client/SparkResult.scala | 0 .../connect/client/arrow/ArrowDeserializer.scala | 0 .../connect/client/arrow/ArrowEncoderUtils.scala | 0 .../sql/connect/client/arrow/ArrowSerializer.scala | 0 .../connect/client/arrow/ArrowVectorReader.scala | 0 .../arrow/ConcatenatingArrowStreamReader.scala | 0 .../apache/spark/sql/connect/client/package.scala | 0 .../spark/sql/connect/client/util/Cleaner.scala| 0 22 files changed, 17 insertions(+), 65 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ClassFinder.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/AmmoniteClassFinder.scala similarity index 58% c
[spark] branch master updated (026aa4fdbdc -> 72a466835a4)
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 026aa4fdbdc [SPARK-43462][SPARK-43871][PS][TESTS] Enable `SeriesDateTimeTests` for pandas 2.0.0 and above add 72a466835a4 [SPARK-44806][CONNECT] Move internal client spark-connect-common to be able to test real in-process server with a real RPC client No new revisions were added by this update. Summary of changes: ...ClassFinder.scala => AmmoniteClassFinder.scala} | 41 ++ connector/connect/common/pom.xml | 12 +++ .../client/arrow/ScalaCollectionUtils.scala| 0 .../client/arrow/ScalaCollectionUtils.scala| 0 .../spark/sql/connect/client/ArtifactManager.scala | 0 .../spark/sql/connect/client/ClassFinder.scala | 27 +- .../sql/connect/client/CloseableIterator.scala | 0 .../client/CustomSparkConnectBlockingStub.scala| 0 .../connect/client/CustomSparkConnectStub.scala| 0 .../ExecutePlanResponseReattachableIterator.scala | 0 .../connect/client/GrpcExceptionConverter.scala| 2 +- .../sql/connect/client/GrpcRetryHandler.scala | 0 .../sql/connect/client/SparkConnectClient.scala| 0 .../connect/client/SparkConnectClientParser.scala | 0 .../spark/sql/connect/client/SparkResult.scala | 0 .../connect/client/arrow/ArrowDeserializer.scala | 0 .../connect/client/arrow/ArrowEncoderUtils.scala | 0 .../sql/connect/client/arrow/ArrowSerializer.scala | 0 .../connect/client/arrow/ArrowVectorReader.scala | 0 .../arrow/ConcatenatingArrowStreamReader.scala | 0 .../apache/spark/sql/connect/client/package.scala | 0 .../spark/sql/connect/client/util/Cleaner.scala| 0 22 files changed, 17 insertions(+), 65 deletions(-) copy connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/{ClassFinder.scala => AmmoniteClassFinder.scala} (58%) rename connector/connect/{client/jvm => common}/src/main/scala-2.12/org/apache/spark/sql/connect/client/arrow/ScalaCollectionUtils.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala-2.13/org/apache/spark/sql/connect/client/arrow/ScalaCollectionUtils.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/ClassFinder.scala (66%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectStub.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala (98%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClientParser.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderUtils.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowVectorReader.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/arrow/ConcatenatingArrowStreamReader.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/package.scala (100%) rename connector/connect/{client/jvm => common}/src/main/scala/org/apache/spark/sql/connect/client/util/Cleaner.scala (100%) - 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-44829][CONNECT] Expose uploadAllArtifactClasses in ArtifactManager to `sql` package
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 c225d4aefef [SPARK-44829][CONNECT] Expose uploadAllArtifactClasses in ArtifactManager to `sql` package c225d4aefef is described below commit c225d4aefef87fd029b0bcbaa23385ce370f0324 Author: Vsevolod Stepanov AuthorDate: Wed Aug 16 21:05:41 2023 +0200 [SPARK-44829][CONNECT] Expose uploadAllArtifactClasses in ArtifactManager to `sql` package ### What changes were proposed in this pull request? This PR exposes `SparkConnectClient`'s `artifactManager.uploadAllArtifactClasses` to `sql` package. ### Why are the changes needed? Currently, UDF classfiles are uploaded in all analyze/execute operations. However, if codepath used to call Scala UDF bypasses we'll need to manually call this method to make sure that all classfiles are uploaded ### How was this patch tested? No tests needed - this PR only exposes a method to sql package Closes #42516 from vsevolodstep-db/SPARK-44829/uploadAllArtifactClasses. Authored-by: Vsevolod Stepanov Signed-off-by: Herman van Hovell (cherry picked from commit 6ae4f4f751fe74944f3acd29c836f0100812117b) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/connect/client/SparkConnectClient.scala| 6 ++ 1 file changed, 6 insertions(+) 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 637499f090c..c41f6dfaae1 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 @@ -62,6 +62,12 @@ private[sql] class SparkConnectClient( new ArtifactManager(configuration, sessionId, bstub, stub) } + /** + * Manually triggers upload of all classfile artifacts to the Spark Connect Server + */ + private[sql] def uploadAllClassFileArtifacts(): Unit = +artifactManager.uploadAllClassFileArtifacts() + /** * Dispatch the [[proto.AnalyzePlanRequest]] to the Spark Connect server. * @return - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-44829][CONNECT] Expose uploadAllArtifactClasses in ArtifactManager to `sql` package
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 6ae4f4f751f [SPARK-44829][CONNECT] Expose uploadAllArtifactClasses in ArtifactManager to `sql` package 6ae4f4f751f is described below commit 6ae4f4f751fe74944f3acd29c836f0100812117b Author: Vsevolod Stepanov AuthorDate: Wed Aug 16 21:05:41 2023 +0200 [SPARK-44829][CONNECT] Expose uploadAllArtifactClasses in ArtifactManager to `sql` package ### What changes were proposed in this pull request? This PR exposes `SparkConnectClient`'s `artifactManager.uploadAllArtifactClasses` to `sql` package. ### Why are the changes needed? Currently, UDF classfiles are uploaded in all analyze/execute operations. However, if codepath used to call Scala UDF bypasses we'll need to manually call this method to make sure that all classfiles are uploaded ### How was this patch tested? No tests needed - this PR only exposes a method to sql package Closes #42516 from vsevolodstep-db/SPARK-44829/uploadAllArtifactClasses. Authored-by: Vsevolod Stepanov Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/connect/client/SparkConnectClient.scala| 6 ++ 1 file changed, 6 insertions(+) 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 637499f090c..c41f6dfaae1 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 @@ -62,6 +62,12 @@ private[sql] class SparkConnectClient( new ArtifactManager(configuration, sessionId, bstub, stub) } + /** + * Manually triggers upload of all classfile artifacts to the Spark Connect Server + */ + private[sql] def uploadAllClassFileArtifacts(): Unit = +artifactManager.uploadAllClassFileArtifacts() + /** * Dispatch the [[proto.AnalyzePlanRequest]] to the Spark Connect server. * @return - 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-44824][CONNECT][TESTS][3.5] Reset `ammoniteOut` in the `afterEach` method of `ReplE2ESuite`
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 02d1f09df0d [SPARK-44824][CONNECT][TESTS][3.5] Reset `ammoniteOut` in the `afterEach` method of `ReplE2ESuite` 02d1f09df0d is described below commit 02d1f09df0da202e3996cdcfbca44525862528b9 Author: yangjie01 AuthorDate: Wed Aug 16 21:03:01 2023 +0200 [SPARK-44824][CONNECT][TESTS][3.5] Reset `ammoniteOut` in the `afterEach` method of `ReplE2ESuite` ### What changes were proposed in this pull request? This PR add `ammoniteOut.reset()` in the `afterEach` method of `ReplE2ESuite` to ensure that the 'output' used for assertions in each test case is only related to the current case and not all content. ### Why are the changes needed? The current `ammoniteOut` records the output content of all executed tests, without isolating between cases. This can lead to unexpected assertion results. For example, adding 'assertContains("""String = "[MyTestClass(1), MyTestClass(3)]"""", output)' in the following test case would still pass the test because it is a result content printed to `ammoniteOut` in the previous test case. https://github.com/apache/spark/blob/2be20e54af6cdf64e8486d1910133b43665f/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala#L283-L290 Hence, we need to clear the content in `ammoniteOut` after each test to achieve isolation between test cases. ### Does this PR introduce _any_ user-facing change? No, just for test ### How was this patch tested? - Pass Github Actions - Manual check Prints the `output` after `val output = runCommandsInShell(input)` in the the case `streaming works with REPL generated code` https://github.com/apache/spark/blob/2be20e54af6cdf64e8486d1910133b43665f/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala#L313-L318 run ``` build/sbt "connect-client-jvm/testOnly org.apache.spark.sql.application.ReplE2ESuite" -Phive ``` **Before**: we can see the content of all test cases that have been executed in the `ReplE2ESuite` ``` Spark session available as 'spark'. _ __ ____ / ___/ __/ /__ / /___ ___ _/ /_ \__ \/ __ \/ __ `/ ___/ //_/ / / / __ \/ __ \/ __ \/ _ \/ ___/ __/ ___/ / /_/ / /_/ / / / ,</ /___/ /_/ / / / / / / / __/ /__/ /_ // .___/\__,_/_/ /_/|_| \/\/_/ /_/_/ /_/\___/\___/\__/ /_/ spark.sql("select 1").collect() res0: Array[org.apache.spark.sql.Row] = Array([1]) semaphore.release() class A(x: Int) { def get = x * 5 + 19 } defined class A def dummyUdf(x: Int): Int = new A(x).get defined function dummyUdf val myUdf = udf(dummyUdf _) myUdf: org.apache.spark.sql.expressions.UserDefinedFunction = ScalarUserDefinedFunction( Array( ... spark.range(5).select(myUdf(col("id"))).as[Int].collect() res5: Array[Int] = Array(19, 24, 29, 34, 39) semaphore.release() class A(x: Int) { def get = x * 42 + 5 } defined class A val myUdf = udf((x: Int) => new A(x).get) myUdf: org.apache.spark.sql.expressions.UserDefinedFunction = ScalarUserDefinedFunction( Array( ... spark.range(5).select(myUdf(col("id"))).as[Int].collect() res9: Array[Int] = Array(5, 47, 89, 131, 173) semaphore.release() class A(x: Int) { def get = x * 7 } defined class A val myUdf = udf((x: Int) => new A(x).get) myUdf: org.apache.spark.sql.expressions.UserDefinedFunction = ScalarUserDefinedFunction( Array( ... val modifiedUdf = myUdf.withName("myUdf").asNondeterministic() modifiedUdf: org.apache.spark.sql.expressions.UserDefinedFunction = ScalarUserDefinedFunction( Array( ... spark.range(5).select(modifiedUdf(col("id"))).as[Int].collect() res14: Array[Int] = Array(0, 7, 14, 21, 28) semaphore.release() spark.range(10).filter(n => n % 2 == 0).collect() res16: Array[java.lang.Long] = Array(0L, 2L, 4L, 6L, 8L) semaphore.release() import java.nio.file.Paths import java.nio.file.Paths def classLoadingTest(x: Int): Int = { val classloader = Option(Thread.currentThread().getContextClassLoader).getOrElse(getClass.getClassLoader) val cls = Class.forName("com.example.Hello$", true, classloader) val module = cls.getField("M
[spark] branch master updated: [SPARK-44807][CONNECT] Add Dataset.metadataColumn to 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 80118e2c688 [SPARK-44807][CONNECT] Add Dataset.metadataColumn to Scala Client 80118e2c688 is described below commit 80118e2c688cdeebf49925385dfec376079b003b Author: Herman van Hovell AuthorDate: Wed Aug 16 18:14:33 2023 +0200 [SPARK-44807][CONNECT] Add Dataset.metadataColumn to Scala Client ### What changes were proposed in this pull request? This PR adds Dataset.metadataColumn to the Spark Connect Scala Client. ### Why are the changes needed? We want the scala client to be as compatible as possible with the API provided by sql/core. ### Does this PR introduce _any_ user-facing change? Yes, it adds a new method to the Spark Connect Scala Client. ### How was this patch tested? I added a test to `ClientE2ETestSuite`. Closes #42492 from hvanhovell/SPARK-44807. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 16 + .../org/apache/spark/sql/ClientE2ETestSuite.scala | 22 +++ .../CheckConnectJvmClientCompatibility.scala | 1 - .../main/protobuf/spark/connect/expressions.proto | 3 + .../sql/connect/planner/SparkConnectPlanner.scala | 3 + .../pyspark/sql/connect/proto/expressions_pb2.py | 68 +++--- .../pyspark/sql/connect/proto/expressions_pb2.pyi | 25 +++- .../catalyst/analysis/ColumnResolutionHelper.scala | 9 ++- .../sql/catalyst/plans/logical/LogicalPlan.scala | 1 + 9 files changed, 111 insertions(+), 37 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 cb7d2c84df5..3c89e649020 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 @@ -1043,6 +1043,22 @@ class Dataset[T] private[sql] ( Column.apply(colName, getPlanId) } + /** + * Selects a metadata column based on its logical column name, and returns it as a [[Column]]. + * + * A metadata column can be accessed this way even if the underlying data source defines a data + * column with a conflicting name. + * + * @group untypedrel + * @since 3.5.0 + */ + def metadataColumn(colName: String): Column = Column { builder => +val attributeBuilder = builder.getUnresolvedAttributeBuilder + .setUnparsedIdentifier(colName) + .setIsMetadataColumn(true) +getPlanId.foreach(attributeBuilder.setPlanId) + } + /** * Selects column based on the column name specified as a regex and returns it as [[Column]]. * @group untypedrel diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index 074cf170dd3..7b9b5f43e80 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -1203,6 +1203,28 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM .dropDuplicatesWithinWatermark("newcol") testAndVerify(result2) } + + test("Dataset.metadataColumn") { +val session: SparkSession = spark +import session.implicits._ +withTempPath { file => + val path = file.getAbsoluteFile.toURI.toString + spark +.range(0, 100, 1, 1) +.withColumn("_metadata", concat(lit("lol_"), col("id"))) +.write +.parquet(file.toPath.toAbsolutePath.toString) + + val df = spark.read.parquet(path) + val (filepath, rc) = df +.groupBy(df.metadataColumn("_metadata").getField("file_path")) +.count() +.as[(String, Long)] +.head() + assert(filepath.startsWith(path)) + assert(rc == 100) +} + } } private[sql] case class ClassData(a: String, b: Int) 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 8f226eb2f7e..3d7a80b1fb6 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 @@ -179,7 +179,6 @@ object CheckConnec
[spark] branch branch-3.5 updated: [SPARK-44799][CONNECT] Fix outer scopes resolution on the executor side
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 a23bb3d0714 [SPARK-44799][CONNECT] Fix outer scopes resolution on the executor side a23bb3d0714 is described below commit a23bb3d07144a85b43160e3e2da2d0bf112d881c Author: Herman van Hovell AuthorDate: Wed Aug 16 15:28:12 2023 +0200 [SPARK-44799][CONNECT] Fix outer scopes resolution on the executor side ### What changes were proposed in this pull request? When you define a class in the REPL (with previously defines symbols), for example: ```scala val filePath = "my_path" case class MyTestClass(value: Int) ``` This is actually declared inside a command class.In ammonite the structure looks like this: ```scala // First command contains the `filePath` object cmd1 { val wrapper = new cmd1 val instance = new command.Helper } class cmd1 extends Serializable { class Helper extends Serializable { val filePath = "my_path" } } // Second contains the `MyTestClass` definition object command2 { val wrapper = new command2 val instance = new command.Helper } class command2 extends Serializable { _root_.scala.transient private val __amm_usedThings = _root_.ammonite.repl.ReplBridge.value.usedEarlierDefinitions.iterator.toSet private val `cmd1`: cmd1.instance.type = if (__amm_usedThings("""cmd1""")) cmd1 else null.asInstanceOf[cmd1.instance.type] class Helper extends Serializable { case class MyTestClass(value: Int) } } ``` In order to create an instance of `MyTestClass` we need an instance of the `Helper`. When an instance of the class is created by Spark itself we use `OuterScopes` that - for Ammonite generated classes - accesses the command object to fetch the helper instance. The problem with this, is that the access triggers the creation of an instance of the command, when you create an instance of the command this tries to access the REPL to figure out which one of its dependents is in use (clever [...] This PR fixes this issue by explicitly passing an getter for the outer instance to the `ProductEncoder`. For ammonite we actually ship the helper instance. This way the encoder always carries the information it needs to create the class. ### Why are the changes needed? This fixes a bug when you try to use a REPL defined class as the input of the UDF. For example this will work now: ```scala val filePath = "my_path" // we need some previous cell that exposes a symbol that could be captured in the class definition. case class MyTestClass(value: Int) { override def toString: String = value.toString } spark.range(10).select(col("id").cast("int").as("value")).as[MyTestClass].map(mtc => mtc.value).collect() ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added a test to `ReplE2ESuite` illustrate the issue. Closes #42489 from hvanhovell/SPARK-44799. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit cb8c653f3c7f76129deae366608613a968b81264) Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 3 +- .../spark/sql/connect/client/SparkResult.scala | 4 +-- .../connect/client/arrow/ArrowDeserializer.scala | 6 ++-- .../sql/connect/client/arrow/ArrowSerializer.scala | 2 +- .../spark/sql/application/ReplE2ESuite.scala | 13 + .../spark/sql/catalyst/ScalaReflection.scala | 5 ++-- .../sql/catalyst/encoders/AgnosticEncoder.scala| 5 ++-- .../spark/sql/catalyst/encoders/OuterScopes.scala | 33 +- .../sql/catalyst/DeserializerBuildHelper.scala | 4 +-- .../spark/sql/catalyst/SerializerBuildHelper.scala | 2 +- .../spark/sql/catalyst/ScalaReflectionSuite.scala | 3 +- 11 files changed, 58 insertions(+), 22 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 28b04fb850e..cb7d2c84df5 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 @@ -883,7 +883,8 @@ class Dataset[T] private[sql] ( ClassTag(SparkClassUtils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple2")), Seq( EncoderField(s"_1", this.agnosticEncoder, leftNullable, Metadata.empty), - EncoderField(s&q
[spark] branch master updated: [SPARK-44799][CONNECT] Fix outer scopes resolution on the executor side
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 cb8c653f3c7 [SPARK-44799][CONNECT] Fix outer scopes resolution on the executor side cb8c653f3c7 is described below commit cb8c653f3c7f76129deae366608613a968b81264 Author: Herman van Hovell AuthorDate: Wed Aug 16 15:28:12 2023 +0200 [SPARK-44799][CONNECT] Fix outer scopes resolution on the executor side ### What changes were proposed in this pull request? When you define a class in the REPL (with previously defines symbols), for example: ```scala val filePath = "my_path" case class MyTestClass(value: Int) ``` This is actually declared inside a command class.In ammonite the structure looks like this: ```scala // First command contains the `filePath` object cmd1 { val wrapper = new cmd1 val instance = new command.Helper } class cmd1 extends Serializable { class Helper extends Serializable { val filePath = "my_path" } } // Second contains the `MyTestClass` definition object command2 { val wrapper = new command2 val instance = new command.Helper } class command2 extends Serializable { _root_.scala.transient private val __amm_usedThings = _root_.ammonite.repl.ReplBridge.value.usedEarlierDefinitions.iterator.toSet private val `cmd1`: cmd1.instance.type = if (__amm_usedThings("""cmd1""")) cmd1 else null.asInstanceOf[cmd1.instance.type] class Helper extends Serializable { case class MyTestClass(value: Int) } } ``` In order to create an instance of `MyTestClass` we need an instance of the `Helper`. When an instance of the class is created by Spark itself we use `OuterScopes` that - for Ammonite generated classes - accesses the command object to fetch the helper instance. The problem with this, is that the access triggers the creation of an instance of the command, when you create an instance of the command this tries to access the REPL to figure out which one of its dependents is in use (clever [...] This PR fixes this issue by explicitly passing an getter for the outer instance to the `ProductEncoder`. For ammonite we actually ship the helper instance. This way the encoder always carries the information it needs to create the class. ### Why are the changes needed? This fixes a bug when you try to use a REPL defined class as the input of the UDF. For example this will work now: ```scala val filePath = "my_path" // we need some previous cell that exposes a symbol that could be captured in the class definition. case class MyTestClass(value: Int) { override def toString: String = value.toString } spark.range(10).select(col("id").cast("int").as("value")).as[MyTestClass].map(mtc => mtc.value).collect() ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added a test to `ReplE2ESuite` illustrate the issue. Closes #42489 from hvanhovell/SPARK-44799. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../main/scala/org/apache/spark/sql/Dataset.scala | 3 +- .../spark/sql/connect/client/SparkResult.scala | 4 +-- .../connect/client/arrow/ArrowDeserializer.scala | 6 ++-- .../sql/connect/client/arrow/ArrowSerializer.scala | 2 +- .../spark/sql/application/ReplE2ESuite.scala | 13 + .../spark/sql/catalyst/ScalaReflection.scala | 5 ++-- .../sql/catalyst/encoders/AgnosticEncoder.scala| 5 ++-- .../spark/sql/catalyst/encoders/OuterScopes.scala | 33 +- .../sql/catalyst/DeserializerBuildHelper.scala | 4 +-- .../spark/sql/catalyst/SerializerBuildHelper.scala | 2 +- .../spark/sql/catalyst/ScalaReflectionSuite.scala | 3 +- 11 files changed, 58 insertions(+), 22 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 28b04fb850e..cb7d2c84df5 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 @@ -883,7 +883,8 @@ class Dataset[T] private[sql] ( ClassTag(SparkClassUtils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple2")), Seq( EncoderField(s"_1", this.agnosticEncoder, leftNullable, Metadata.empty), - EncoderField(s"_2", other.agnosticEncoder, rightNullable, Metadata.empty))) + EncoderField(s"_2", other.a
[spark] branch branch-3.5 updated: [SPARK-42664][CONNECT] Support `bloomFilter` function for `DataFrameStatFunctions`
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 a4bd7583ce6 [SPARK-42664][CONNECT] Support `bloomFilter` function for `DataFrameStatFunctions` a4bd7583ce6 is described below commit a4bd7583ce6d680f0091519007e48894d594b9f6 Author: yangjie01 AuthorDate: Tue Aug 15 19:12:03 2023 +0200 [SPARK-42664][CONNECT] Support `bloomFilter` function for `DataFrameStatFunctions` ### What changes were proposed in this pull request? This is pr using `BloomFilterAggregate` to implement `bloomFilter` function for `DataFrameStatFunctions`. ### Why are the changes needed? Add Spark connect jvm client api coverage. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Add new test - Manually check Scala 2.13 Closes #42414 from LuciferYang/SPARK-42664-backup. Authored-by: yangjie01 Signed-off-by: Herman van Hovell (cherry picked from commit b9f11143d058ad05dcda2138133471c9500c8b92) Signed-off-by: Herman van Hovell --- .../org/apache/spark/util/sketch/BloomFilter.java | 4 +- .../apache/spark/sql/DataFrameStatFunctions.scala | 88 +- .../spark/sql/ClientDataFrameStatSuite.scala | 87 + .../CheckConnectJvmClientCompatibility.scala | 3 - .../sql/connect/planner/SparkConnectPlanner.scala | 31 .../aggregate/BloomFilterAggregate.scala | 43 ++- 6 files changed, 248 insertions(+), 8 deletions(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java index 5c01841e501..f3c2b05e7af 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java @@ -199,9 +199,9 @@ public abstract class BloomFilter { * See http://en.wikipedia.org/wiki/Bloom_filter#Probability_of_false_positives for the formula. * * @param n expected insertions (must be positive) - * @param p false positive rate (must be 0 < p < 1) + * @param p false positive rate (must be 0 p 1) */ - private static long optimalNumOfBits(long n, double p) { + public static long optimalNumOfBits(long n, double p) { return (long) (-n * Math.log(p) / (Math.log(2) * Math.log(2))); } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index 0d4372b8738..4d35b4e8767 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -18,14 +18,16 @@ package org.apache.spark.sql import java.{lang => jl, util => ju} +import java.io.ByteArrayInputStream import scala.collection.JavaConverters._ +import org.apache.spark.SparkException import org.apache.spark.connect.proto.{Relation, StatSampleBy} import org.apache.spark.sql.DataFrameStatFunctions.approxQuantileResultEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, PrimitiveDoubleEncoder} import org.apache.spark.sql.functions.lit -import org.apache.spark.util.sketch.CountMinSketch +import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch} /** * Statistic functions for `DataFrame`s. @@ -584,6 +586,90 @@ final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, roo } CountMinSketch.readFrom(ds.head()) } + + /** + * Builds a Bloom filter over a specified column. + * + * @param colName + * name of the column over which the filter is built + * @param expectedNumItems + * expected number of items which will be put into the filter. + * @param fpp + * expected false positive probability of the filter. + * @since 3.5.0 + */ + def bloomFilter(colName: String, expectedNumItems: Long, fpp: Double): BloomFilter = { +buildBloomFilter(Column(colName), expectedNumItems, -1L, fpp) + } + + /** + * Builds a Bloom filter over a specified column. + * + * @param col + * the column over which the filter is built + * @param expectedNumItems + * expected number of items which will be put into the filter. + * @param fpp + * expected false positive probability of the filter. + * @since 3.5.0 + */ + def bloomFilter(col: Column, expectedNumItems: Long, fpp: Double): BloomFilter = { +buildBloomFilter(col, expectedNumItems, -1L, fpp) + } + + /** + * Builds a Bloom filter over a specified column. + * + * @p
[spark] branch master updated (2ab404ff179 -> b9f11143d05)
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 2ab404ff179 [SPARK-44794][CONNECT] Make Streaming Queries work with Connect's artifact management add b9f11143d05 [SPARK-42664][CONNECT] Support `bloomFilter` function for `DataFrameStatFunctions` No new revisions were added by this update. Summary of changes: .../org/apache/spark/util/sketch/BloomFilter.java | 4 +- .../apache/spark/sql/DataFrameStatFunctions.scala | 88 +- .../spark/sql/ClientDataFrameStatSuite.scala | 87 + .../CheckConnectJvmClientCompatibility.scala | 3 - .../sql/connect/planner/SparkConnectPlanner.scala | 31 .../aggregate/BloomFilterAggregate.scala | 43 ++- 6 files changed, 248 insertions(+), 8 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-44794][CONNECT] Make Streaming Queries work with Connect's artifact management
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 78a6b46b5f8 [SPARK-44794][CONNECT] Make Streaming Queries work with Connect's artifact management 78a6b46b5f8 is described below commit 78a6b46b5f87f7a4b86e1d2ce010069a9027bdf9 Author: Herman van Hovell AuthorDate: Tue Aug 15 19:02:31 2023 +0200 [SPARK-44794][CONNECT] Make Streaming Queries work with Connect's artifact management ### What changes were proposed in this pull request? When you try to run a streaming query that contains a UDF from the REPL, for example: ```scala val add1 = udf((i: Long) => i + 1) val query = spark.readStream .format("rate") .option("rowsPerSecond", "10") .option("numPartitions", "1") .load() .withColumn("value", add1($"value")) .writeStream .format("memory") .queryName("my_sink") .start() ``` You are currently greeted by a hard to understand deserialization issue, where a serialization proxy cannot be assigned to a field. The underlying cause here is a `ClassNotFoundException` (yes, java serialization is weird). This `ClassNotFoundException` is caused by us not propagating the `JobArtifactState` (this - indirectly - contains information about the location of REPL generated classes, and session local libraries) properly to the streaming query execution thread. This PR fixed this by propagating the `JobArtifactState` into the stream execution thread. ### Why are the changes needed? It is a bug. We want streaming to work with connect's isolated dependencies. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I added a test to `ReplE2ESuite`. Closes #42476 from hvanhovell/SPARK-44794. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 2ab404ff179ae04e86d4441fa8c28c08f95e0d0f) Signed-off-by: Herman van Hovell --- .../spark/sql/application/ReplE2ESuite.scala | 28 ++ .../sql/execution/streaming/StreamExecution.scala | 9 +-- 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala index 191fd851482..13ca5caf0af 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala @@ -288,4 +288,32 @@ class ReplE2ESuite extends RemoteSparkSession with BeforeAndAfterEach { val output = runCommandsInShell(input) assertContains("Array[MyTestClass] = Array(MyTestClass(0), MyTestClass(1))", output) } + + test("streaming works with REPL generated code") { +val input = + """ +|val add1 = udf((i: Long) => i + 1) +|val query = { +| spark.readStream +| .format("rate") +| .option("rowsPerSecond", "10") +| .option("numPartitions", "1") +| .load() +| .withColumn("value", add1($"value")) +| .writeStream +| .format("memory") +| .queryName("my_sink") +| .start() +|} +|var progress = query.lastProgress +|while (query.isActive && (progress == null || progress.numInputRows == 0)) { +| query.awaitTermination(100) +| progress = query.lastProgress +|} +|val noException = query.exception.isEmpty +|query.stop() +|""".stripMargin +val output = runCommandsInShell(input) +assertContains("noException: Boolean = true", output) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 88fb28f0da4..936de41af76 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -31,7 +31,7 @@ import scala.util.control.NonFatal import com.google.common.util.concurrent.UncheckedExecutionException import org.apache.hadoop.fs.Path -import org.apache.spark.{SparkContext, SparkException, SparkThrowable} +import org.apache.spark.{JobArti
[spark] branch master updated (afe188687ba -> 2ab404ff179)
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 afe188687ba [SPARK-44803][BUILD] Replace `publish` with `publishOrSkip` in SparkBuild to eliminate warnings add 2ab404ff179 [SPARK-44794][CONNECT] Make Streaming Queries work with Connect's artifact management No new revisions were added by this update. Summary of changes: .../spark/sql/application/ReplE2ESuite.scala | 28 ++ .../sql/execution/streaming/StreamExecution.scala | 9 +-- 2 files changed, 35 insertions(+), 2 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-44795][CONNECT] CodeGenerator Cache should be classloader specific
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 76e61c44aa2 [SPARK-44795][CONNECT] CodeGenerator Cache should be classloader specific 76e61c44aa2 is described below commit 76e61c44aa28e12d363d86a401fd581bf3b054c1 Author: Herman van Hovell AuthorDate: Tue Aug 15 03:10:42 2023 +0200 [SPARK-44795][CONNECT] CodeGenerator Cache should be classloader specific ### What changes were proposed in this pull request? When you currently use a REPL generated class in a UDF you can get an error saying that that class is not equal to that class. This error is thrown in a code generated class. The problem is that the classes have been loaded by different classloaders. We cache generated code and use the textual code as the string. The problem with this is that in Spark Connect users are free in supplying user classes that can have arbitrary names, a name can point to an entirely different class, or it [...] There are roughly two ways how this problem can arise: 1. Two sessions use the same class names. This is particularly easy when you use the REPL because this always generates the same names. 2. You run in single process mode. In this case wholestage codegen will test compile the class using a different classloader then the 'executor', while sharing the same code generator cache. ### Why are the changes needed? We want to be able to use REPL (and other) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I added a test to the `ReplE2ESuite`. Closes #42478 from hvanhovell/SPARK-44795. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit 1d562904e4e75aec3ea8d4999ede0183fda326c7) Signed-off-by: Herman van Hovell --- .../spark/sql/application/ReplE2ESuite.scala | 9 .../spark/sql/catalyst/encoders/OuterScopes.scala | 49 +- .../expressions/codegen/CodeGenerator.scala| 30 ++--- 3 files changed, 54 insertions(+), 34 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala index 0e69b5afa45..0cab66eef3d 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala @@ -276,4 +276,13 @@ class ReplE2ESuite extends RemoteSparkSession with BeforeAndAfterEach { val output = runCommandsInShell(input) assertContains("Array[MyTestClass] = Array(MyTestClass(1), MyTestClass(3))", output) } + + test("REPL class in UDF") { +val input = """ +|case class MyTestClass(value: Int) +|spark.range(2).map(i => MyTestClass(i.toInt)).collect() + """.stripMargin +val output = runCommandsInShell(input) +assertContains("Array[MyTestClass] = Array(MyTestClass(0), MyTestClass(1))", output) + } } 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` ->
[spark] branch master updated: [SPARK-44795][CONNECT] CodeGenerator Cache should be classloader specific
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 1d562904e4e [SPARK-44795][CONNECT] CodeGenerator Cache should be classloader specific 1d562904e4e is described below commit 1d562904e4e75aec3ea8d4999ede0183fda326c7 Author: Herman van Hovell AuthorDate: Tue Aug 15 03:10:42 2023 +0200 [SPARK-44795][CONNECT] CodeGenerator Cache should be classloader specific ### What changes were proposed in this pull request? When you currently use a REPL generated class in a UDF you can get an error saying that that class is not equal to that class. This error is thrown in a code generated class. The problem is that the classes have been loaded by different classloaders. We cache generated code and use the textual code as the string. The problem with this is that in Spark Connect users are free in supplying user classes that can have arbitrary names, a name can point to an entirely different class, or it [...] There are roughly two ways how this problem can arise: 1. Two sessions use the same class names. This is particularly easy when you use the REPL because this always generates the same names. 2. You run in single process mode. In this case wholestage codegen will test compile the class using a different classloader then the 'executor', while sharing the same code generator cache. ### Why are the changes needed? We want to be able to use REPL (and other) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I added a test to the `ReplE2ESuite`. Closes #42478 from hvanhovell/SPARK-44795. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../spark/sql/application/ReplE2ESuite.scala | 9 .../spark/sql/catalyst/encoders/OuterScopes.scala | 49 +- .../expressions/codegen/CodeGenerator.scala| 30 ++--- 3 files changed, 54 insertions(+), 34 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala index 0e69b5afa45..0cab66eef3d 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala @@ -276,4 +276,13 @@ class ReplE2ESuite extends RemoteSparkSession with BeforeAndAfterEach { val output = runCommandsInShell(input) assertContains("Array[MyTestClass] = Array(MyTestClass(1), MyTestClass(3))", output) } + + test("REPL class in UDF") { +val input = """ +|case class MyTestClass(value: Int) +|spark.range(2).map(i => MyTestClass(i.toInt)).collect() + """.stripMargin +val output = runCommandsInShell(input) +assertContains("Array[MyTestClass] = Array(MyTestClass(0), MyTestClass(1))", output) + } } 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\.cm
[spark] 03/03: Revert "Add test"
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 481546385a605143d3a6103fa2aaf9936612465d Author: Herman van Hovell AuthorDate: Mon Aug 14 05:39:15 2023 +0200 Revert "Add test" This reverts commit 2c465f681ada8be8cb53edfff3ddbd273b89bc72. --- .../scala/org/apache/spark/sql/application/ReplE2ESuite.scala| 9 - 1 file changed, 9 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala index 0cab66eef3d..0e69b5afa45 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala @@ -276,13 +276,4 @@ class ReplE2ESuite extends RemoteSparkSession with BeforeAndAfterEach { val output = runCommandsInShell(input) assertContains("Array[MyTestClass] = Array(MyTestClass(1), MyTestClass(3))", output) } - - test("REPL class in UDF") { -val input = """ -|case class MyTestClass(value: Int) -|spark.range(2).map(i => MyTestClass(i.toInt)).collect() - """.stripMargin -val output = runCommandsInShell(input) -assertContains("Array[MyTestClass] = Array(MyTestClass(0), MyTestClass(1))", output) - } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] 02/03: Revert "Add classloader Id to code generation cache."
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 3c1ea33a3bfd5c355fff635499c48905b24ca3e8 Author: Herman van Hovell AuthorDate: Mon Aug 14 05:39:06 2023 +0200 Revert "Add classloader Id to code generation cache." This reverts commit 6d4891b32cee585523a51a5304d6aa3c47bb8af8. --- .../expressions/codegen/CodeGenerator.scala| 40 -- 1 file changed, 14 insertions(+), 26 deletions(-) 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..8d10f6cd295 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 @@ -1441,7 +1439,7 @@ object CodeGenerator extends Logging { * @return a pair of a generated class and the bytecode statistics of generated functions. */ def compile(code: CodeAndComment): (GeneratedClass, ByteCodeStats) = try { -cache.get((classLoaderUUID.get(Utils.getContextOrSparkClassLoader), code)) +cache.get(code) } catch { // Cache.get() may wrap the original exception. See the following URL // https://guava.dev/releases/14.0.1/api/docs/com/google/common/cache/ @@ -1583,30 +1581,20 @@ object CodeGenerator extends Logging { * aborted. See [[NonFateSharingCache]] for more details. */ private val cache = { -val loadFunc: ((ClassLoaderId, CodeAndComment)) => (GeneratedClass, ByteCodeStats) = { - case (_, code) => -val startTime = System.nanoTime() -val result = doCompile(code) -val endTime = System.nanoTime() -val duration = endTime - startTime -val timeMs: Double = duration.toDouble / NANOS_PER_MILLIS -CodegenMetrics.METRIC_SOURCE_CODE_SIZE.update(code.body.length) -CodegenMetrics.METRIC_COMPILATION_TIME.update(timeMs.toLong) -logInfo(s"Code generated in $timeMs ms") -_compileTime.add(duration) -result +def loadFunc: CodeAndComment => (GeneratedClass, ByteCodeStats) = code => { + val startTime = System.nanoTime() + val result = doCompile(code) + val endTime = System.nanoTime() + val duration = endTime - startTime + val timeMs: Double = duration.toDouble / NANOS_PER_MILLIS + CodegenMetrics.METRIC_SOURCE_CODE_SIZE.update(code.body.length) + CodegenMetrics.METRIC_COMPILATION_TIME.update(timeMs.toLong) + logInfo(s"Code generated in $timeMs ms") + _compileTime.add(duration) + result } -NonFateSharingCache(loadFunc, SQLConf.get.codegenCacheMaxEntries) - } - - type ClassLoaderId = String - private val classLoaderUUID = { -NonFateSharingCache(CacheBuilder.newBuilder() - .weakKeys - .maximumSize(SQLConf.get.codegenCacheMaxEntries) - .build(new CacheLoader[ClassLoader, ClassLoaderId]() { -override def load(code: ClassLoader): ClassLoaderId = UUID.randomUUID.toString - })) +NonFateSharingCache[CodeAndComment, (GeneratedClass, ByteCodeStats)]( + loadFunc, SQLConf.get.codegenCacheMaxEntries) } /** - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (890748873bd -> 481546385a6)
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 890748873bd Refine solution new 8a0e0591c83 Revert "Refine solution" new 3c1ea33a3bf Revert "Add classloader Id to code generation cache." new 481546385a6 Revert "Add test" The 3 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../spark/sql/application/ReplE2ESuite.scala | 9 .../spark/sql/catalyst/encoders/OuterScopes.scala | 49 +- .../expressions/codegen/CodeGenerator.scala| 30 +++-- 3 files changed, 34 insertions(+), 54 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] 01/03: Revert "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 8a0e0591c83f450b75b0c066ba50fc2d3f20b290 Author: Herman van Hovell AuthorDate: Mon Aug 14 05:38:54 2023 +0200 Revert "Refine solution" This reverts commit 890748873bd8bd72b34d3f907ecdb72a694234c9. --- .../spark/sql/catalyst/encoders/OuterScopes.scala | 49 +- .../expressions/codegen/CodeGenerator.scala| 18 ++-- 2 files changed, 34 insertions(+), 33 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 6c10e8ece80..c2ac504c846 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,9 +26,28 @@ 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, queue) +new HashableWeakReference(c.getClassLoader) } private[this] val outerScopes = { @@ -135,31 +154,3 @@ 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 fe61cc81359..59688cae889 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,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import java.io.ByteArrayInputStream +import java.util.UUID import scala.annotation.tailrec import scala.collection.JavaConverters._ @@ -25,6 +26,7 @@ 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 @@ -35,7 +37,6 @@ 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.ap
[spark] 01/03: Add test
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 2c465f681ada8be8cb53edfff3ddbd273b89bc72 Author: Herman van Hovell AuthorDate: Mon Aug 14 04:59:59 2023 +0200 Add test --- .../scala/org/apache/spark/sql/application/ReplE2ESuite.scala| 9 + 1 file changed, 9 insertions(+) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala index 0e69b5afa45..0cab66eef3d 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala @@ -276,4 +276,13 @@ class ReplE2ESuite extends RemoteSparkSession with BeforeAndAfterEach { val output = runCommandsInShell(input) assertContains("Array[MyTestClass] = Array(MyTestClass(1), MyTestClass(3))", output) } + + test("REPL class in UDF") { +val input = """ +|case class MyTestClass(value: Int) +|spark.range(2).map(i => MyTestClass(i.toInt)).collect() + """.stripMargin +val output = runCommandsInShell(input) +assertContains("Array[MyTestClass] = Array(MyTestClass(0), MyTestClass(1))", output) + } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (6bd95d0e004 -> 890748873bd)
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 6bd95d0e004 [SPARK-44775][PYTHON][DOCS] Add missing version information in DataFrame APIs new 2c465f681ad Add test new 6d4891b32ce Add classloader Id to code generation cache. new 890748873bd Refine solution The 3 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../spark/sql/application/ReplE2ESuite.scala | 9 .../spark/sql/catalyst/encoders/OuterScopes.scala | 49 +- .../expressions/codegen/CodeGenerator.scala| 30 ++--- 3 files changed, 54 insertions(+), 34 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] 02/03: Add classloader Id to code generation cache.
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 6d4891b32cee585523a51a5304d6aa3c47bb8af8 Author: Herman van Hovell AuthorDate: Sun Aug 13 02:49:19 2023 +0200 Add classloader Id to code generation cache. --- .../expressions/codegen/CodeGenerator.scala| 40 ++ 1 file changed, 26 insertions(+), 14 deletions(-) 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 8d10f6cd295..59688cae889 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,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import java.io.ByteArrayInputStream +import java.util.UUID import scala.annotation.tailrec import scala.collection.JavaConverters._ @@ -25,6 +26,7 @@ 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 @@ -1439,7 +1441,7 @@ object CodeGenerator extends Logging { * @return a pair of a generated class and the bytecode statistics of generated functions. */ def compile(code: CodeAndComment): (GeneratedClass, ByteCodeStats) = try { -cache.get(code) +cache.get((classLoaderUUID.get(Utils.getContextOrSparkClassLoader), code)) } catch { // Cache.get() may wrap the original exception. See the following URL // https://guava.dev/releases/14.0.1/api/docs/com/google/common/cache/ @@ -1581,20 +1583,30 @@ object CodeGenerator extends Logging { * aborted. See [[NonFateSharingCache]] for more details. */ private val cache = { -def loadFunc: CodeAndComment => (GeneratedClass, ByteCodeStats) = code => { - val startTime = System.nanoTime() - val result = doCompile(code) - val endTime = System.nanoTime() - val duration = endTime - startTime - val timeMs: Double = duration.toDouble / NANOS_PER_MILLIS - CodegenMetrics.METRIC_SOURCE_CODE_SIZE.update(code.body.length) - CodegenMetrics.METRIC_COMPILATION_TIME.update(timeMs.toLong) - logInfo(s"Code generated in $timeMs ms") - _compileTime.add(duration) - result +val loadFunc: ((ClassLoaderId, CodeAndComment)) => (GeneratedClass, ByteCodeStats) = { + case (_, code) => +val startTime = System.nanoTime() +val result = doCompile(code) +val endTime = System.nanoTime() +val duration = endTime - startTime +val timeMs: Double = duration.toDouble / NANOS_PER_MILLIS +CodegenMetrics.METRIC_SOURCE_CODE_SIZE.update(code.body.length) +CodegenMetrics.METRIC_COMPILATION_TIME.update(timeMs.toLong) +logInfo(s"Code generated in $timeMs ms") +_compileTime.add(duration) +result } -NonFateSharingCache[CodeAndComment, (GeneratedClass, ByteCodeStats)]( - loadFunc, SQLConf.get.codegenCacheMaxEntries) +NonFateSharingCache(loadFunc, SQLConf.get.codegenCacheMaxEntries) + } + + type ClassLoaderId = String + private val classLoaderUUID = { +NonFateSharingCache(CacheBuilder.newBuilder() + .weakKeys + .maximumSize(SQLConf.get.codegenCacheMaxEntries) + .build(new CacheLoader[ClassLoader, ClassLoaderId]() { +override def load(code: ClassLoader): ClassLoaderId = UUID.randomUUID.toString + })) } /** - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[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.catalyst.type
[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
[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 reattachable execution, or parts the
[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. // Non reattacha
[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.getSubqueryAlia
[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
[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
[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 100644 ---
[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 SparkSession
[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 creati
[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[_] if
[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[_] if
[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 - -// sequence
[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