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: [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]

Reply via email to