You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "rangadi (via GitHub)" <gi...@apache.org> on 2023/07/17 07:00:50 UTC

[GitHub] [spark] rangadi opened a new pull request, #42035: [SPARK-42944] Streaming ForeachBatch

rangadi opened a new pull request, #42035:
URL: https://github.com/apache/spark/pull/42035

   Adds `foreachBatch()` in Python. This adds a new runner `StreamingPythonRunner`.
   Note that this PR focuses on core functionality and includes TODO for followup improvements (will update with jira tickets where missing).
   
   Included more inline comments to help with the review. 
   
   ### What changes were proposed in this pull request?
   Adds support for foreachBatch() in Spark connect. 
   
   ### Why are the changes needed?
     - Manual tests
     - Unit tests:
        - The tests are updated to use a global temp view, rather than shared variable since connect version of the function runs on the server side.
   
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] rangadi commented on a diff in pull request #42035: [SPARK-42944] Streaming ForeachBatch in Python

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1265804405


##########
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:
   Added tickets. 



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala:
##########
@@ -57,13 +63,59 @@ 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) => {
+        fn(args.df, args.batchId) // dfId is not used, see hack comment above.
+      },
+      sessionHolder)
   }
+
+  def pythonForeachBatchWrapper(

Review Comment:
   Done.



##########
core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala:
##########
@@ -110,6 +110,57 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
     }
   }
 
+  def createStreamingWorker(): (Socket, Option[Int]) = {

Review Comment:
   Drastically simplified this by reusing existing `createSimpleWorker()`. 
   Added brief documentation. 



##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.api.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+import java.net.Socket
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.BUFFER_SIZE
+import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTHON_USE_DAEMON}
+
+
+private[spark] object StreamingPythonRunner {
+  def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner = {
+    new StreamingPythonRunner(func, connectUrl)
+  }
+}
+
+private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: String)
+  extends Logging {
+  private val conf = SparkEnv.get.conf
+  protected val bufferSize: Int = conf.get(BUFFER_SIZE)
+  protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  protected val pythonVer: String = func.pythonVer
+
+  def init(sessionId: String): (DataOutputStream, DataInputStream) = {

Review Comment:
   Done.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on pull request #42035: [SPARK-42944][SS][PYTHON] Streaming ForeachBatch in Python

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on PR #42035:
URL: https://github.com/apache/spark/pull/42035#issuecomment-1641145068

   Let me merge first. Please make a followup PR to address my comments.
   
   Merged to master and branch-3.5.


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #42035: [SPARK-42944][SS][PYTHON] Streaming ForeachBatch in Python

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1266398022


##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.api.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+import java.net.Socket
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.BUFFER_SIZE
+import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTHON_USE_DAEMON}
+
+
+private[spark] object StreamingPythonRunner {
+  def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner = {
+    new StreamingPythonRunner(func, connectUrl)
+  }
+}
+
+private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: String)
+  extends Logging {
+  private val conf = SparkEnv.get.conf
+  protected val bufferSize: Int = conf.get(BUFFER_SIZE)
+  protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  protected val pythonVer: String = func.pythonVer
+
+  /**
+   * Initializes the Python worker for streaming functions. Sets up Spark Connect session
+   * to be used with the functions.
+   */
+  def init(sessionId: String): (DataOutputStream, DataInputStream) = {
+    log.info(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")
+
+    val env = SparkEnv.get
+
+    val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",")
+    envVars.put("SPARK_LOCAL_DIRS", localdir)
+
+    envVars.put("SPARK_AUTH_SOCKET_TIMEOUT", authSocketTimeout.toString)
+    envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString)
+    conf.set(PYTHON_USE_DAEMON, false)
+    envVars.put("SPARK_CONNECT_LOCAL_URL", connectUrl)
+
+    val pythonWorkerFactory = new PythonWorkerFactory(pythonExec, envVars.asScala.toMap)
+    val (worker: Socket, _) = pythonWorkerFactory.createStreamingWorker()

Review Comment:
   I am fine with doing it in a followup.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] rangadi commented on pull request #42035: [SPARK-42944] Streaming ForeachBatch

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on PR #42035:
URL: https://github.com/apache/spark/pull/42035#issuecomment-1637490391

   cc: @bogao007, @WweiL, @HyukjinKwon PTAL.


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] rangadi commented on a diff in pull request #42035: [SPARK-42944] Streaming ForeachBatch in Python

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1265651351


##########
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:
   Thanks. Just realized that minutes ago :). Fixing. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] bogao007 commented on a diff in pull request #42035: [SPARK-42944] Streaming ForeachBatch in Python

Posted by "bogao007 (via GitHub)" <gi...@apache.org>.
bogao007 commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1265682022


##########
core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala:
##########
@@ -110,6 +110,57 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
     }
   }
 
+  def createStreamingWorker(): (Socket, Option[Int]) = {

Review Comment:
   Nit: Can we add some documentations for this method?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala:
##########
@@ -57,13 +63,59 @@ 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) => {
+        fn(args.df, args.batchId) // dfId is not used, see hack comment above.
+      },
+      sessionHolder)
   }
+
+  def pythonForeachBatchWrapper(

Review Comment:
   Nit: Missing documentation



##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.api.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+import java.net.Socket
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.BUFFER_SIZE
+import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTHON_USE_DAEMON}
+
+
+private[spark] object StreamingPythonRunner {
+  def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner = {
+    new StreamingPythonRunner(func, connectUrl)
+  }
+}
+
+private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: String)
+  extends Logging {
+  private val conf = SparkEnv.get.conf
+  protected val bufferSize: Int = conf.get(BUFFER_SIZE)
+  protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  protected val pythonVer: String = func.pythonVer
+
+  def init(sessionId: String): (DataOutputStream, DataInputStream) = {

Review Comment:
   Nit: Some documentations for this method would be good



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] rangadi commented on a diff in pull request #42035: [SPARK-42944] Streaming ForeachBatch in Python

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1265621046


##########
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:
   cc: @WweiL. Do you any reason why we are not using connect session here?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] rangadi commented on a diff in pull request #42035: [SPARK-42944][SS][PYTHON] Streaming ForeachBatch in Python

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1266100223


##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.api.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+import java.net.Socket
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.BUFFER_SIZE
+import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTHON_USE_DAEMON}
+
+
+private[spark] object StreamingPythonRunner {
+  def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner = {
+    new StreamingPythonRunner(func, connectUrl)
+  }
+}
+
+private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: String)
+  extends Logging {
+  private val conf = SparkEnv.get.conf
+  protected val bufferSize: Int = conf.get(BUFFER_SIZE)
+  protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  protected val pythonVer: String = func.pythonVer
+
+  /**
+   * Initializes the Python worker for streaming functions. Sets up Spark Connect session
+   * to be used with the functions.
+   */
+  def init(sessionId: String): (DataOutputStream, DataInputStream) = {
+    log.info(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")
+
+    val env = SparkEnv.get
+
+    val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",")
+    envVars.put("SPARK_LOCAL_DIRS", localdir)
+
+    envVars.put("SPARK_AUTH_SOCKET_TIMEOUT", authSocketTimeout.toString)
+    envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString)
+    conf.set(PYTHON_USE_DAEMON, false)
+    envVars.put("SPARK_CONNECT_LOCAL_URL", connectUrl)
+
+    val pythonWorkerFactory = new PythonWorkerFactory(pythonExec, envVars.asScala.toMap)
+    val (worker: Socket, _) = pythonWorkerFactory.createStreamingWorker()

Review Comment:
   What we are doing here is also as simple: 
   ```scala
   def createStreamingWorker(): (Socket, Option[Int]) = {
       createSimpleWorker("pyspark.streaming_worker")
   }
   ```
   But we can change in #41948 to your suggestion when you are merging it.
   



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #42035: [SPARK-42944][SS][PYTHON] Streaming ForeachBatch in Python

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1267397837


##########
python/pyspark/sql/connect/session.py:
##########
@@ -654,6 +655,14 @@ def copyFromLocalToFs(self, local_path: str, dest_path: str) -> None:
 
     copyFromLocalToFs.__doc__ = PySparkSession.copyFromLocalToFs.__doc__
 
+    def _createRemoteDataFrame(self, remote_id: str) -> "DataFrame":

Review Comment:
   ```suggestion
       def _create_remote_dataframe(self, remote_id: str) -> "DataFrame":
   ```



##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    write_int,
+    read_long,
+    UTF8Deserializer,
+    CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile):  # type: ignore[no-untyped-def]
+    log_name = "Streaming ForeachBatch worker"
+    connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+    sessionId = utf8_deserializer.loads(infile)
+
+    print(f"{log_name} is starting with url {connect_url} and sessionId {sessionId}.")
+
+    sparkConnectSession = SparkSession.builder.remote(connect_url).getOrCreate()

Review Comment:
   ```suggestion
       spark_connect_session = SparkSession.builder.remote(connect_url).getOrCreate()
   ```



##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.api.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+import java.net.Socket
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.BUFFER_SIZE
+import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTHON_USE_DAEMON}
+
+
+private[spark] object StreamingPythonRunner {
+  def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner = {
+    new StreamingPythonRunner(func, connectUrl)
+  }
+}
+
+private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: String)
+  extends Logging {
+  private val conf = SparkEnv.get.conf
+  protected val bufferSize: Int = conf.get(BUFFER_SIZE)
+  protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  protected val pythonVer: String = func.pythonVer
+
+  /**
+   * Initializes the Python worker for streaming functions. Sets up Spark Connect session
+   * to be used with the functions.
+   */
+  def init(sessionId: String): (DataOutputStream, DataInputStream) = {
+    log.info(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")

Review Comment:
   ```suggestion
       logInfo(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")
   ```



##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    write_int,
+    read_long,
+    UTF8Deserializer,
+    CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile):  # type: ignore[no-untyped-def]
+    log_name = "Streaming ForeachBatch worker"
+    connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+    sessionId = utf8_deserializer.loads(infile)
+
+    print(f"{log_name} is starting with url {connect_url} and sessionId {sessionId}.")
+
+    sparkConnectSession = SparkSession.builder.remote(connect_url).getOrCreate()
+    sparkConnectSession._client._session_id = sessionId
+
+    # TODO(SPARK-44460): Pass credentials.
+    # TODO(SPARK-44461): Enable Process Isolation
+
+    func = worker.read_command(pickleSer, infile)

Review Comment:
   ```suggestion
       func = worker.read_command(pickle_ser, infile)
   ```



##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    write_int,
+    read_long,
+    UTF8Deserializer,
+    CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()

Review Comment:
   ```suggestion
   pickle_ser = CPickleSerializer()
   ```



##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    write_int,
+    read_long,
+    UTF8Deserializer,
+    CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile):  # type: ignore[no-untyped-def]
+    log_name = "Streaming ForeachBatch worker"
+    connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+    sessionId = utf8_deserializer.loads(infile)

Review Comment:
   ```suggestion
       session_id = utf8_deserializer.loads(infile)
   ```



##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    write_int,
+    read_long,
+    UTF8Deserializer,
+    CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile):  # type: ignore[no-untyped-def]
+    log_name = "Streaming ForeachBatch worker"
+    connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+    sessionId = utf8_deserializer.loads(infile)
+
+    print(f"{log_name} is starting with url {connect_url} and sessionId {sessionId}.")
+
+    sparkConnectSession = SparkSession.builder.remote(connect_url).getOrCreate()
+    sparkConnectSession._client._session_id = sessionId
+
+    # TODO(SPARK-44460): Pass credentials.
+    # TODO(SPARK-44461): Enable Process Isolation
+
+    func = worker.read_command(pickleSer, infile)
+    write_int(0, outfile)  # Indicate successful initialization
+
+    outfile.flush()
+
+    def process(dfId, batchId):  # type: ignore[no-untyped-def]
+        print(f"{log_name} Started batch {batchId} with DF id {dfId}")
+        batchDf = sparkConnectSession._createRemoteDataFrame(dfId)
+        func(batchDf, batchId)

Review Comment:
   ```suggestion
           func(batch_df, batchId)
   ```



##########
python/pyspark/sql/connect/streaming/readwriter.py:
##########
@@ -495,14 +495,14 @@ def foreach(self, f: Union[Callable[[Row], None], "SupportsProcess"]) -> "DataSt
 
     foreach.__doc__ = PySparkDataStreamWriter.foreach.__doc__
 
-    # TODO (SPARK-42944): Implement and uncomment the doc
     def foreachBatch(self, func: Callable[["DataFrame", int], None]) -> "DataStreamWriter":
-        raise PySparkNotImplementedError(
-            error_class="NOT_IMPLEMENTED",
-            message_parameters={"feature": "foreachBatch()"},
+        self._write_proto.foreach_batch.python_function.command = CloudPickleSerializer().dumps(
+            func
         )
+        self._write_proto.foreach_batch.python_function.python_ver = "%d.%d" % sys.version_info[:2]
+        return self
 
-    # foreachBatch.__doc__ = PySparkDataStreamWriter.foreachBatch.__doc__
+    foreachBatch.__doc__ = PySparkDataStreamWriter.foreachBatch.__doc__

Review Comment:
   We should fix non-Spark Connect foreachBatch docs and add a directive like:
   
   ```    .. versionchanged:: 3.4.0
           Supports Spark Connect.
   
   
   ```



##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    write_int,
+    read_long,
+    UTF8Deserializer,
+    CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile):  # type: ignore[no-untyped-def]
+    log_name = "Streaming ForeachBatch worker"
+    connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+    sessionId = utf8_deserializer.loads(infile)
+
+    print(f"{log_name} is starting with url {connect_url} and sessionId {sessionId}.")
+
+    sparkConnectSession = SparkSession.builder.remote(connect_url).getOrCreate()
+    sparkConnectSession._client._session_id = sessionId

Review Comment:
   ```suggestion
       sparkConnectSession._client._session_id = session_id
   ```



##########
python/pyspark/streaming_worker.py:
##########
@@ -0,0 +1,78 @@
+#
+# 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.
+#
+
+"""
+A worker for streaming foreachBatch and query listener in Spark Connect.
+"""
+import os
+
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    write_int,
+    read_long,
+    UTF8Deserializer,
+    CPickleSerializer,
+)
+from pyspark import worker
+from pyspark.sql import SparkSession
+
+pickleSer = CPickleSerializer()
+utf8_deserializer = UTF8Deserializer()
+
+
+def main(infile, outfile):  # type: ignore[no-untyped-def]
+    log_name = "Streaming ForeachBatch worker"
+    connect_url = os.environ["SPARK_CONNECT_LOCAL_URL"]
+    sessionId = utf8_deserializer.loads(infile)
+
+    print(f"{log_name} is starting with url {connect_url} and sessionId {sessionId}.")
+
+    sparkConnectSession = SparkSession.builder.remote(connect_url).getOrCreate()
+    sparkConnectSession._client._session_id = sessionId
+
+    # TODO(SPARK-44460): Pass credentials.
+    # TODO(SPARK-44461): Enable Process Isolation
+
+    func = worker.read_command(pickleSer, infile)
+    write_int(0, outfile)  # Indicate successful initialization
+
+    outfile.flush()
+
+    def process(dfId, batchId):  # type: ignore[no-untyped-def]
+        print(f"{log_name} Started batch {batchId} with DF id {dfId}")
+        batchDf = sparkConnectSession._createRemoteDataFrame(dfId)

Review Comment:
   ```suggestion
           batch_df = spark_connect_session._create_remote_dataframe(dfId)
   ```



##########
python/pyspark/sql/tests/connect/streaming/test_parity_streaming_foreachBatch.py:
##########
@@ -0,0 +1,44 @@
+#

Review Comment:
   should add this into `dev/sparktestsupport/modules.py` so the tests actually run



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] rangadi commented on a diff in pull request #42035: [SPARK-42944] Streaming ForeachBatch

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #42035: [SPARK-42944][SS][PYTHON] Streaming ForeachBatch in Python

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1265999659


##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.api.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+import java.net.Socket
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.BUFFER_SIZE
+import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTHON_USE_DAEMON}
+
+
+private[spark] object StreamingPythonRunner {
+  def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner = {
+    new StreamingPythonRunner(func, connectUrl)
+  }
+}
+
+private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: String)
+  extends Logging {
+  private val conf = SparkEnv.get.conf
+  protected val bufferSize: Int = conf.get(BUFFER_SIZE)
+  protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  protected val pythonVer: String = func.pythonVer
+
+  /**
+   * Initializes the Python worker for streaming functions. Sets up Spark Connect session
+   * to be used with the functions.
+   */
+  def init(sessionId: String): (DataOutputStream, DataInputStream) = {
+    log.info(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")
+
+    val env = SparkEnv.get
+
+    val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",")
+    envVars.put("SPARK_LOCAL_DIRS", localdir)
+
+    envVars.put("SPARK_AUTH_SOCKET_TIMEOUT", authSocketTimeout.toString)
+    envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString)
+    conf.set(PYTHON_USE_DAEMON, false)
+    envVars.put("SPARK_CONNECT_LOCAL_URL", connectUrl)
+
+    val pythonWorkerFactory = new PythonWorkerFactory(pythonExec, envVars.asScala.toMap)
+    val (worker: Socket, _) = pythonWorkerFactory.createStreamingWorker()

Review Comment:
   @ueshin is actually working on this in https://github.com/apache/spark/pull/41948. I believe we can reuse the same codebase.



##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.api.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+import java.net.Socket
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.BUFFER_SIZE
+import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTHON_USE_DAEMON}
+
+
+private[spark] object StreamingPythonRunner {
+  def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner = {
+    new StreamingPythonRunner(func, connectUrl)
+  }
+}
+
+private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: String)
+  extends Logging {
+  private val conf = SparkEnv.get.conf
+  protected val bufferSize: Int = conf.get(BUFFER_SIZE)
+  protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  protected val pythonVer: String = func.pythonVer
+
+  /**
+   * Initializes the Python worker for streaming functions. Sets up Spark Connect session
+   * to be used with the functions.
+   */
+  def init(sessionId: String): (DataOutputStream, DataInputStream) = {
+    log.info(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")
+
+    val env = SparkEnv.get
+
+    val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",")
+    envVars.put("SPARK_LOCAL_DIRS", localdir)
+
+    envVars.put("SPARK_AUTH_SOCKET_TIMEOUT", authSocketTimeout.toString)
+    envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString)
+    conf.set(PYTHON_USE_DAEMON, false)
+    envVars.put("SPARK_CONNECT_LOCAL_URL", connectUrl)
+
+    val pythonWorkerFactory = new PythonWorkerFactory(pythonExec, envVars.asScala.toMap)
+    val (worker: Socket, _) = pythonWorkerFactory.createStreamingWorker()

Review Comment:
   The logic here and there are virtually same .. if I am not wrong.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] WweiL commented on a diff in pull request #42035: [SPARK-42944] Streaming ForeachBatch in Python

Posted by "WweiL (via GitHub)" <gi...@apache.org>.
WweiL commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1265648967


##########
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:
   I think it's because there is a typo here here:
   ```
   class StreamingTestsForeachBatchMixin(ReusedSQLTestCase):
   ```
   You don't want the mixin class to extend any class. The spark session is very likely set there



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] rangadi commented on a diff in pull request #42035: [SPARK-42944][SS][PYTHON] Streaming ForeachBatch in Python

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1266009168


##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.api.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+import java.net.Socket
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.BUFFER_SIZE
+import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTHON_USE_DAEMON}
+
+
+private[spark] object StreamingPythonRunner {
+  def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner = {
+    new StreamingPythonRunner(func, connectUrl)
+  }
+}
+
+private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: String)
+  extends Logging {
+  private val conf = SparkEnv.get.conf
+  protected val bufferSize: Int = conf.get(BUFFER_SIZE)
+  protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  protected val pythonVer: String = func.pythonVer
+
+  /**
+   * Initializes the Python worker for streaming functions. Sets up Spark Connect session
+   * to be used with the functions.
+   */
+  def init(sessionId: String): (DataOutputStream, DataInputStream) = {
+    log.info(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")
+
+    val env = SparkEnv.get
+
+    val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",")
+    envVars.put("SPARK_LOCAL_DIRS", localdir)
+
+    envVars.put("SPARK_AUTH_SOCKET_TIMEOUT", authSocketTimeout.toString)
+    envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString)
+    conf.set(PYTHON_USE_DAEMON, false)
+    envVars.put("SPARK_CONNECT_LOCAL_URL", connectUrl)
+
+    val pythonWorkerFactory = new PythonWorkerFactory(pythonExec, envVars.asScala.toMap)
+    val (worker: Socket, _) = pythonWorkerFactory.createStreamingWorker()

Review Comment:
   Yeah, we can share. From my first look #41948 does not seem to initialize a session. Also, it is not a long running process with multiple iterations like here.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] ueshin commented on a diff in pull request #42035: [SPARK-42944][SS][PYTHON] Streaming ForeachBatch in Python

Posted by "ueshin (via GitHub)" <gi...@apache.org>.
ueshin commented on code in PR #42035:
URL: https://github.com/apache/spark/pull/42035#discussion_r1266075072


##########
core/src/main/scala/org/apache/spark/api/python/StreamingPythonRunner.scala:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.api.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+import java.net.Socket
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.BUFFER_SIZE
+import org.apache.spark.internal.config.Python.{PYTHON_AUTH_SOCKET_TIMEOUT, PYTHON_USE_DAEMON}
+
+
+private[spark] object StreamingPythonRunner {
+  def apply(func: PythonFunction, connectUrl: String): StreamingPythonRunner = {
+    new StreamingPythonRunner(func, connectUrl)
+  }
+}
+
+private[spark] class StreamingPythonRunner(func: PythonFunction, connectUrl: String)
+  extends Logging {
+  private val conf = SparkEnv.get.conf
+  protected val bufferSize: Int = conf.get(BUFFER_SIZE)
+  protected val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  protected val pythonVer: String = func.pythonVer
+
+  /**
+   * Initializes the Python worker for streaming functions. Sets up Spark Connect session
+   * to be used with the functions.
+   */
+  def init(sessionId: String): (DataOutputStream, DataInputStream) = {
+    log.info(s"Initializing Python runner (session: $sessionId ,pythonExec: $pythonExec")
+
+    val env = SparkEnv.get
+
+    val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",")
+    envVars.put("SPARK_LOCAL_DIRS", localdir)
+
+    envVars.put("SPARK_AUTH_SOCKET_TIMEOUT", authSocketTimeout.toString)
+    envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString)
+    conf.set(PYTHON_USE_DAEMON, false)
+    envVars.put("SPARK_CONNECT_LOCAL_URL", connectUrl)
+
+    val pythonWorkerFactory = new PythonWorkerFactory(pythonExec, envVars.asScala.toMap)
+    val (worker: Socket, _) = pythonWorkerFactory.createStreamingWorker()

Review Comment:
   How about:
   
   ```scala
   val pythonWorkerFactory = env.createPythonWorker(pythonExec, "pyspark.streaming_worker", envVars.asScala.toMap)
   val (worker: Socket, _) = pythonWorkerFactory.createSimpleWorker()
   ```
   
   based on #41948 with making `private[spark] createSimpleWorker`.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon closed pull request #42035: [SPARK-42944][SS][PYTHON] Streaming ForeachBatch in Python

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon closed pull request #42035: [SPARK-42944][SS][PYTHON] Streaming ForeachBatch in Python
URL: https://github.com/apache/spark/pull/42035


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org