rangadi commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1264951053
##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala:
##########
@@ -57,13 +63,63 @@ object StreamingForeachBatchHelper extends Logging {
* Handles setting up Scala remote session and other Spark Connect
environment and then runs the
* provided foreachBatch function `fn`.
*
- * HACK ALERT: This version does not atually set up Spark connect. Directly
passes the
- * DataFrame, so the user code actually runs with legacy DataFrame.
+ * HACK ALERT: This version does not actually set up Spark Connect session.
Directly passes the
+ * DataFrame, so the user code actually runs with legacy DataFrame and
session..
*/
def scalaForeachBatchWrapper(
fn: ForeachBatchFnType,
sessionHolder: SessionHolder): ForeachBatchFnType = {
// TODO: Set up Spark Connect session. Do we actually need this for the
first version?
- dataFrameCachingWrapper(fn, sessionHolder)
+ dataFrameCachingWrapper(
+ (args: FnArgsWithId) => {
+ assert(sessionHolder.session == args.df.sparkSession) // XXX
+ fn(args.df, args.batchId) // dfId is not used, see hack comment above.
+ },
+ sessionHolder)
+ }
+
+ def pythonForeachBatchWrapper(
+ pythonFn: SimplePythonFunction,
+ sessionHolder: SessionHolder): ForeachBatchFnType = {
+
+ val port = SparkConnectService.localPort
+ val connectUrl = s"sc://localhost:$port/;user_id=${sessionHolder.userId}"
+ val runner = StreamingPythonRunner(pythonFn, connectUrl)
+ val (dataOut, dataIn) = runner.init(sessionHolder.sessionId)
+
+ val foreachBatchRunnerFn: FnArgsWithId => Unit = (args: FnArgsWithId) => {
+
+ // TODO: Set userId
+ // TODO: Auth credentials
+ // TODO: The current protocol is very basic. Improve this, especially
for SafeSpark.
+
+ // TODO: A new session id pointing to args.df.sparKSEssion needs to be
created.
+ // This is because MicroBatch execution clones the session during
start.
+ // The session attached to the foreachBatch dataframe is different
from the one the one
+ // the query was started with.
+
+ PythonRDD.writeUTF(args.dfId, dataOut)
+ dataOut.writeLong(args.batchId)
+ dataOut.flush()
+
+ val ret = dataIn.readInt()
+ log.info(s"Python foreach batch for dfId ${args.dfId} completed (ret:
$ret)")
+
+ // When to terminate the runner: See comment below.
+ // TODO: What does daemon process mean in this context? Do we need it?
+ }
+
+ dataFrameCachingWrapper(foreachBatchRunnerFn, sessionHolder)
}
+
+ // TODO(SPARK-44433): Improve termination of Processes
Review Comment:
Termination logic will be added as an immediate follow up to this PR.
##########
python/pyspark/sql/tests/connect/streaming/test_parity_streaming_foreachBatch.py:
##########
@@ -0,0 +1,36 @@
+#
+# 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.
+#
+
+from pyspark.sql.tests.streaming.test_streaming_foreachBatch import
StreamingTestsForeachBatchMixin
+from pyspark.testing.connectutils import ReusedConnectTestCase
+
+
+class StreamingForeachBatchParityTests(StreamingTestsForeachBatchMixin,
ReusedConnectTestCase):
+ pass
+
+
+if __name__ == "__main__":
+ import unittest
+ from
pyspark.sql.tests.connect.streaming.test_parity_streaming_foreachBatch import
StreamingForeachBatchParityTests # noqa: F401
Review Comment:
@HyukjinKwon I have updated the tests, but still this one seems to be
running non-connect tests. Do you see anything odd set up with class?
##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala:
##########
@@ -57,13 +63,63 @@ object StreamingForeachBatchHelper extends Logging {
* Handles setting up Scala remote session and other Spark Connect
environment and then runs the
* provided foreachBatch function `fn`.
*
- * HACK ALERT: This version does not atually set up Spark connect. Directly
passes the
- * DataFrame, so the user code actually runs with legacy DataFrame.
+ * HACK ALERT: This version does not actually set up Spark Connect session.
Directly passes the
+ * DataFrame, so the user code actually runs with legacy DataFrame and
session..
*/
def scalaForeachBatchWrapper(
fn: ForeachBatchFnType,
sessionHolder: SessionHolder): ForeachBatchFnType = {
// TODO: Set up Spark Connect session. Do we actually need this for the
first version?
- dataFrameCachingWrapper(fn, sessionHolder)
+ dataFrameCachingWrapper(
+ (args: FnArgsWithId) => {
+ assert(sessionHolder.session == args.df.sparkSession) // XXX
+ fn(args.df, args.batchId) // dfId is not used, see hack comment above.
+ },
+ sessionHolder)
+ }
+
+ def pythonForeachBatchWrapper(
+ pythonFn: SimplePythonFunction,
+ sessionHolder: SessionHolder): ForeachBatchFnType = {
+
+ val port = SparkConnectService.localPort
+ val connectUrl = s"sc://localhost:$port/;user_id=${sessionHolder.userId}"
+ val runner = StreamingPythonRunner(pythonFn, connectUrl)
+ val (dataOut, dataIn) = runner.init(sessionHolder.sessionId)
+
+ val foreachBatchRunnerFn: FnArgsWithId => Unit = (args: FnArgsWithId) => {
+
+ // TODO: Set userId
+ // TODO: Auth credentials
Review Comment:
Will file tickets for the TODOs and update here.
##########
python/pyspark/sql/tests/streaming/test_streaming_foreachBatch.py:
##########
@@ -20,40 +20,41 @@
from pyspark.testing.sqlutils import ReusedSQLTestCase
-class StreamingTestsForeachBatch(ReusedSQLTestCase):
+class StreamingTestsForeachBatchMixin(ReusedSQLTestCase):
def test_streaming_foreachBatch(self):
q = None
- collected = dict()
def collectBatch(batch_df, batch_id):
- collected[batch_id] = batch_df.collect()
+ batch_df.createOrReplaceGlobalTempView("test_view")
Review Comment:
Using global temp-view rather than a global variable to verify.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]