You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "chaoqin-li1123 (via GitHub)" <gi...@apache.org> on 2024/02/04 21:24:23 UTC

[PR] [SPARK-46962[SS] Implement python worker to run python streaming data source [spark]

chaoqin-li1123 opened a new pull request, #45023:
URL: https://github.com/apache/spark/pull/45023

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'common/utils/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   Implement python worker to run python streaming data source and communicate with JVM through socket. Create a PythonMicrobatchStream to invoke RPC function call.
   
   
   ### Why are the changes needed?
   In preparation for support of development of streaming data source in Python.
   
   
   ### Does this PR introduce _any_ user-facing change?
   No.
   
   
   ### How was this patch tested?
   Unit test.
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   No.
   


-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1480622631


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonScan.scala:
##########
@@ -45,13 +63,50 @@ class PythonScan(
     new PythonPartitionReaderFactory(
       ds.source, readerFunc, outputSchema, jobArtifactUUID)
   }
+}
 
-  override def toBatch: Batch = this
+case class PythonStreamingSourceOffset(json: String) extends Offset
 
-  override def description: String = "(Python)"
+case class PythonStreamingSourcePartition(partition: Array[Byte]) extends InputPartition
 
-  override def readSchema(): StructType = outputSchema
+class PythonMicroBatchStream(

Review Comment:
   should we put this in a different file? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val partitionsFuncId = 886
+  val latestOffsetsFuncId = 887

Review Comment:
   what are these hardcoded numbers?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonScan.scala:
##########
@@ -27,9 +30,24 @@ class PythonScan(
      ds: PythonDataSourceV2,
      shortName: String,
      outputSchema: StructType,
-     options: CaseInsensitiveStringMap) extends Batch with Scan {
+     options: CaseInsensitiveStringMap) extends Scan {
 
-  private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid)
+  override def toBatch: Batch = new PythonBatch(ds, shortName, outputSchema, options)
+
+  override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream =
+    new PythonMicroBatchStream(ds, shortName, outputSchema, options)
+
+  override def description: String = "(Python)"
+
+  override def readSchema(): StructType = outputSchema
+
+  override def supportedCustomMetrics(): Array[CustomMetric] =
+    ds.source.createPythonMetrics()
+}
+
+class PythonBatch(ds: PythonDataSourceV2, shortName: String,
+                  outputSchema: StructType, options: CaseInsensitiveStringMap) extends Batch {

Review Comment:
   nit: ident



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val partitionsFuncId = 886
+  val latestOffsetsFuncId = 887
+}
+
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for streaming functions. Sets up Spark Connect session

Review Comment:
   Where is the Spark Connect session?



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   We expect data source to know about its offset model - we pass a json format of offset to specific data source, which is expected to be deserialized successfully to its offset model. It will come to the runtime error if the json format of offset and offset model aren't compatible, which I think Scala side is doing the same.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  // When the python process for python_streaming_source_runner receives one of the
+  // integers below, it will invoke the corresponding function of StreamReader instance.
+  val INITIAL_OFFSET_FUNC_ID = 884
+  val LATEST_OFFSET_FUNC_ID = 885
+  val PARTITIONS_FUNC_ID = 886
+  val COMMIT_FUNC_ID = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  private val conf = SparkEnv.get.conf
+  private val bufferSize: Int = conf.get(BUFFER_SIZE)
+  private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  private val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(LATEST_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(INITIAL_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(PARTITIONS_FUNC_ID)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "planPartitions", msg)
+    }
+    for (_ <- 0 until numPartitions) {
+      val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn)
+      pickledPartitions.append(pickledPartition)
+    }
+    pickledPartitions.toArray
+  }
+
+  /**
+   * Invokes commit(end) function of the stream reader and receive the return value.
+   */
+  def commit(end: String): Unit = {
+    dataOut.writeInt(COMMIT_FUNC_ID)

Review Comment:
   I'm wondering if the python process somehow dies, would this (and other read/write functions) throw an error? 
   
   In UDF runners I believe the job just aborted and restarted so should be fine. But on driver, would this be an issue? Should we generally do try-catch here or somewhere in upstream logic? cc @chaoqin-li1123 @HyukjinKwon @allisonwang-db 



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1520229021


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.

Review Comment:
   It is just discouraged, list and float are also supposed to work.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1517356072


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def initialOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def latestOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod
+    def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:

Review Comment:
   I copy the interface from batch python data source.  Why is this Iterator[Union] instead of Union[Iterator]? @HyukjinKwon 



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1491663766


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod
+    def read(self, partition) -> Iterator[Union[Tuple, Row]]:
+        """
+        Generates data for a given partition and returns an iterator of tuples or rows.
+
+        This method is invoked once per partition to read the data. Implementing
+        this method is required for stream reader. You can initialize any
+        non-serializable resources required for reading data from the data source
+        within this method.
+        This method is static and stateless. You shouldn't access mutable class member
+        or keep in memory state between different invocations of read().
+
+        Parameters
+        ----------
+        partition : object
+            The partition to read. It must be one of the partition values returned by
+            ``partitions()``.
+
+        Returns
+        -------
+        Iterator[Tuple] or Iterator[Row]
+            An iterator of tuples or rows. Each tuple or row will be converted to a row
+            in the final DataFrame.
+        """
+        ...
+
+    def commit(self, end: dict):

Review Comment:
   Return type added.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   I just realized this is too limited - there are data sources which has an offset structure as complex model, e.g. offset model consists of a list containing case class instance as element. It probably couldn't be bound to flattened dictionary.
   
   Shall we revisit the design for the structure of offset? Probably the closest approach from proposed change to do is to simply allow dict, with noticing that we are serializing dict to json via following approach underneath (hence the dict has to be compatible). 
   https://docs.python.org/3/library/json.html#py-to-json-table
   
   Or just introduce an interface for Offset which is expected to handle json serde by itself. We could probably document the example; how to implement json serde for the complex type of offset.



##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,178 @@
+#
+# 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.
+#
+
+import os
+import sys
+import functools
+import json
+from itertools import islice
+from typing import IO, List, Iterator, Iterable
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.connect.conversion import ArrowTableToRowsConversion, LocalDataToArrowConversion
+from pyspark.sql.datasource import DataSource, InputPartition
+from pyspark.sql.pandas.types import to_arrow_schema
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    BinaryType,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader, outfile):
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader, outfile):
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader, infile, outfile):
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader, infile, outfile):
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)
+        schema = _parse_datatype_json_string(schema_json)
+        if not isinstance(schema, StructType):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "an output schema of type 'StructType'",
+                    "actual": f"'{type(schema).__name__}'",
+                },
+            )
+
+        # Receive the configuration values.
+        max_arrow_batch_size = read_int(infile)
+        assert max_arrow_batch_size > 0, (
+            "The maximum arrow batch size should be greater than 0, but got "
+            f"'{max_arrow_batch_size}'"
+        )
+
+        # Instantiate data source reader.
+        try:
+            reader = data_source.streamReader(schema=schema)
+            # Initialization succeed.
+            write_int(0, outfile)
+            outfile.flush()
+
+            # handle method call from socket
+            while True:
+                func_id = read_int(infile)
+                if func_id == initial_offset_func_id:
+                    initial_offset_func(reader, outfile)
+                elif func_id == latest_offset_func_id:
+                    latest_offset_func(reader, outfile)
+                elif func_id == partitions_func_id:
+                    partitions_func(reader, infile, outfile)
+                elif func_id == commit_func_id:
+                    commit_func(reader, infile, outfile)
+                outfile.flush()
+
+        except NotImplementedError:
+            raise PySparkRuntimeError(
+                error_class="PYTHON_DATA_SOURCE_METHOD_NOT_IMPLEMENTED",
+                message_parameters={"type": "reader", "method": "reader"},
+            )
+        except Exception as e:

Review Comment:
   Isn't this too broad compared to what we record as? If we want to classify it as "data source creation error", I think we should narrow the scope to the call of streamReader, not beyond.
   
   Please look back how batch side does this.



##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,178 @@
+#
+# 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.
+#
+
+import os
+import sys
+import functools
+import json
+from itertools import islice
+from typing import IO, List, Iterator, Iterable
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.connect.conversion import ArrowTableToRowsConversion, LocalDataToArrowConversion
+from pyspark.sql.datasource import DataSource, InputPartition
+from pyspark.sql.pandas.types import to_arrow_schema
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    BinaryType,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader, outfile):
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader, outfile):
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader, infile, outfile):
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader, infile, outfile):
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)
+        schema = _parse_datatype_json_string(schema_json)
+        if not isinstance(schema, StructType):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "an output schema of type 'StructType'",
+                    "actual": f"'{type(schema).__name__}'",
+                },
+            )
+
+        # Receive the configuration values.

Review Comment:
   This does not seem to be used - will this be referred in later part?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)

Review Comment:
   @HyukjinKwon Do you anticipate any issue with temporary changing the conf value and reverting back?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.execution.python
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs}
+import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase {
+
+  protected def simpleDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class SimpleDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        return {"0": "2"}
+      |    def latestOffset(self):
+      |        return {"0": "2"}
+      |    def partitions(self, start: dict, end: dict):
+      |        return [InputPartition(i) for i in range(int(start["0"]))]
+      |    def commit(self, end: dict):
+      |        1 + 2
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  protected def errorDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class ErrorDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        raise Exception("error reading initial offset")
+      |    def latestOffset(self):
+      |        raise Exception("error reading latest offset")
+      |    def partitions(self, start: dict, end: dict):
+      |        raise Exception("error planning partitions")
+      |    def commit(self, end: dict):
+      |        raise Exception("error committing offset")
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  private val errorDataSourceName = "ErrorDataSource"
+
+  test("simple data stream source") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |$simpleDataStreamReaderScript
+         |
+         |class $dataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        return SimpleDataStreamReader()
+         |""".stripMargin
+    val inputSchema = StructType.fromDDL("input BINARY")
+
+    val dataSource = createUserDefinedPythonDataSource(dataSourceName, dataSourceScript)
+    spark.dataSource.registerPython(dataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("SimpleDataSource")
+    val stream = new PythonMicroBatchStream(
+      pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+
+    val initialOffset = stream.initialOffset()
+    assert(initialOffset.json == "{\"0\": \"2\"}")
+    for (_ <- 1 to 50) {
+      val offset = stream.latestOffset()
+      assert(offset.json == "{\"0\": \"2\"}")
+      assert(stream.planInputPartitions(offset, offset).size == 2)
+      stream.commit(offset)
+    }
+    stream.stop()
+  }
+
+  test("Error creating stream reader") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |class $dataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        raise Exception("error creating stream reader")
+         |""".stripMargin
+    val dataSource = createUserDefinedPythonDataSource(
+      name = dataSourceName, pythonScript = dataSourceScript)
+    spark.dataSource.registerPython(dataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("SimpleDataSource")
+    val inputSchema = StructType.fromDDL("input BINARY")
+    val err = intercept[AnalysisException] {
+      new PythonMicroBatchStream(
+        pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+    }
+    assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR")

Review Comment:
   If the error is bound to error class framework, you need to verify with checkError.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.execution.python
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs}
+import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase {
+
+  protected def simpleDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class SimpleDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        return {"0": "2"}
+      |    def latestOffset(self):
+      |        return {"0": "2"}
+      |    def partitions(self, start: dict, end: dict):
+      |        return [InputPartition(i) for i in range(int(start["0"]))]

Review Comment:
   What is `start["0"]`?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.execution.python
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs}
+import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase {
+
+  protected def simpleDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class SimpleDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        return {"0": "2"}
+      |    def latestOffset(self):
+      |        return {"0": "2"}
+      |    def partitions(self, start: dict, end: dict):
+      |        return [InputPartition(i) for i in range(int(start["0"]))]
+      |    def commit(self, end: dict):
+      |        1 + 2
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  protected def errorDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class ErrorDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        raise Exception("error reading initial offset")
+      |    def latestOffset(self):
+      |        raise Exception("error reading latest offset")
+      |    def partitions(self, start: dict, end: dict):
+      |        raise Exception("error planning partitions")
+      |    def commit(self, end: dict):
+      |        raise Exception("error committing offset")
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  private val errorDataSourceName = "ErrorDataSource"
+
+  test("simple data stream source") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |$simpleDataStreamReaderScript
+         |
+         |class $dataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        return SimpleDataStreamReader()
+         |""".stripMargin
+    val inputSchema = StructType.fromDDL("input BINARY")
+
+    val dataSource = createUserDefinedPythonDataSource(dataSourceName, dataSourceScript)
+    spark.dataSource.registerPython(dataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("SimpleDataSource")
+    val stream = new PythonMicroBatchStream(
+      pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+
+    val initialOffset = stream.initialOffset()
+    assert(initialOffset.json == "{\"0\": \"2\"}")
+    for (_ <- 1 to 50) {
+      val offset = stream.latestOffset()
+      assert(offset.json == "{\"0\": \"2\"}")
+      assert(stream.planInputPartitions(offset, offset).size == 2)
+      stream.commit(offset)
+    }
+    stream.stop()
+  }
+
+  test("Error creating stream reader") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |class $dataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        raise Exception("error creating stream reader")
+         |""".stripMargin
+    val dataSource = createUserDefinedPythonDataSource(
+      name = dataSourceName, pythonScript = dataSourceScript)
+    spark.dataSource.registerPython(dataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("SimpleDataSource")
+    val inputSchema = StructType.fromDDL("input BINARY")
+    val err = intercept[AnalysisException] {
+      new PythonMicroBatchStream(
+        pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+    }
+    assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR")
+    assert(err.getMessage.contains("error creating stream reader"))
+  }
+
+  test("Error in stream reader") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |$errorDataStreamReaderScript
+         |
+         |class $errorDataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        return ErrorDataStreamReader()
+         |""".stripMargin
+    val inputSchema = StructType.fromDDL("input BINARY")
+
+    val dataSource = createUserDefinedPythonDataSource(errorDataSourceName, dataSourceScript)
+    spark.dataSource.registerPython(errorDataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("ErrorDataSource")
+    val offset = PythonStreamingSourceOffset("{\"0\": \"2\"}")
+
+    def testMicroBatchStreamError(msg: String)
+                                 (func: PythonMicroBatchStream => Unit): Unit = {
+      val stream = new PythonMicroBatchStream(
+        pythonDs, errorDataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+      val err = intercept[SparkException] {
+        func(stream)
+      }
+      assert(err.getMessage.contains(msg))

Review Comment:
   If the error is bound to error class framework, you need to verify with checkError.



##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,178 @@
+#
+# 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.
+#
+
+import os
+import sys
+import functools
+import json
+from itertools import islice
+from typing import IO, List, Iterator, Iterable
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.connect.conversion import ArrowTableToRowsConversion, LocalDataToArrowConversion
+from pyspark.sql.datasource import DataSource, InputPartition
+from pyspark.sql.pandas.types import to_arrow_schema
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    BinaryType,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader, outfile):
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader, outfile):
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader, infile, outfile):
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader, infile, outfile):
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)

Review Comment:
   What is the expected behavior if user does not specify the schema and relies on the schema of data source (which is mostly applied to the majority of data sources)?



##########
python/pyspark/sql/datasource.py:
##########
@@ -159,6 +160,27 @@ def writer(self, schema: StructType, overwrite: bool) -> "DataSourceWriter":
             message_parameters={"feature": "writer"},
         )
 
+    def streamReader(self, schema: StructType) -> "DataSourceStreamReader":
+        """
+        Returns a ``DataSourceStreamReader`` instance for reading streaming data.
+
+        The implementation is required for readable streaming data sources.
+
+        Parameters
+        ----------
+        schema : StructType
+            The schema of the data to be written.

Review Comment:
   This seems to be copied incorrectly from batch writer. Should be from batch reader.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonMicroBatchStream.scala:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.execution.datasources.v2.python
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory}
+import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset}
+import org.apache.spark.sql.execution.python.PythonStreamingSourceRunner
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+case class PythonStreamingSourceOffset(json: String) extends Offset
+
+case class PythonStreamingSourcePartition(partition: Array[Byte]) extends InputPartition
+
+class PythonMicroBatchStream(
+    ds: PythonDataSourceV2,
+    shortName: String,
+    outputSchema: StructType,
+    options: CaseInsensitiveStringMap
+  ) extends MicroBatchStream with Logging {
+  private def createDataSourceFunc =
+    ds.source.createPythonFunction(
+      ds.getOrCreateDataSourceInPython(shortName, options, Some(outputSchema)).dataSource)
+
+  val runner: PythonStreamingSourceRunner =

Review Comment:
   Does this need to be public?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonMicroBatchStream.scala:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.execution.datasources.v2.python
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory}
+import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset}
+import org.apache.spark.sql.execution.python.PythonStreamingSourceRunner
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+case class PythonStreamingSourceOffset(json: String) extends Offset
+
+case class PythonStreamingSourcePartition(partition: Array[Byte]) extends InputPartition
+
+class PythonMicroBatchStream(
+    ds: PythonDataSourceV2,
+    shortName: String,
+    outputSchema: StructType,
+    options: CaseInsensitiveStringMap
+  ) extends MicroBatchStream with Logging {
+  private def createDataSourceFunc =
+    ds.source.createPythonFunction(
+      ds.getOrCreateDataSourceInPython(shortName, options, Some(outputSchema)).dataSource)
+
+  val runner: PythonStreamingSourceRunner =
+    new PythonStreamingSourceRunner(createDataSourceFunc, outputSchema)
+  runner.init()
+
+  override def initialOffset(): Offset = PythonStreamingSourceOffset(runner.initialOffset())
+
+  override def latestOffset(): Offset = PythonStreamingSourceOffset(runner.latestOffset())
+

Review Comment:
   nit: unnecessary two empty lines, remove one



##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,178 @@
+#
+# 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.
+#
+
+import os
+import sys
+import functools
+import json
+from itertools import islice
+from typing import IO, List, Iterator, Iterable
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.connect.conversion import ArrowTableToRowsConversion, LocalDataToArrowConversion
+from pyspark.sql.datasource import DataSource, InputPartition
+from pyspark.sql.pandas.types import to_arrow_schema
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    BinaryType,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader, outfile):
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader, outfile):
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader, infile, outfile):
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader, infile, outfile):
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)
+        schema = _parse_datatype_json_string(schema_json)
+        if not isinstance(schema, StructType):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "an output schema of type 'StructType'",
+                    "actual": f"'{type(schema).__name__}'",
+                },
+            )
+
+        # Receive the configuration values.
+        max_arrow_batch_size = read_int(infile)
+        assert max_arrow_batch_size > 0, (
+            "The maximum arrow batch size should be greater than 0, but got "
+            f"'{max_arrow_batch_size}'"
+        )
+
+        # Instantiate data source reader.
+        try:
+            reader = data_source.streamReader(schema=schema)
+            # Initialization succeed.
+            write_int(0, outfile)
+            outfile.flush()
+
+            # handle method call from socket
+            while True:
+                func_id = read_int(infile)
+                if func_id == initial_offset_func_id:
+                    initial_offset_func(reader, outfile)
+                elif func_id == latest_offset_func_id:
+                    latest_offset_func(reader, outfile)
+                elif func_id == partitions_func_id:
+                    partitions_func(reader, infile, outfile)
+                elif func_id == commit_func_id:
+                    commit_func(reader, infile, outfile)
+                outfile.flush()
+
+        except NotImplementedError:
+            raise PySparkRuntimeError(
+                error_class="PYTHON_DATA_SOURCE_METHOD_NOT_IMPLEMENTED",
+                message_parameters={"type": "reader", "method": "reader"},

Review Comment:
   Shouldn't we provide the actual method name to be missed?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    // Send configurations
+    dataOut.writeInt(SQLConf.get.arrowMaxRecordsPerBatch)
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(latestOffsetFuncId)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(

Review Comment:
   Shall we check whether this error message is enough to identify which source has an issue with among multiple sources?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonMicroBatchStream.scala:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.execution.datasources.v2.python
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory}
+import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset}
+import org.apache.spark.sql.execution.python.PythonStreamingSourceRunner
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+case class PythonStreamingSourceOffset(json: String) extends Offset
+
+case class PythonStreamingSourcePartition(partition: Array[Byte]) extends InputPartition
+
+class PythonMicroBatchStream(
+    ds: PythonDataSourceV2,
+    shortName: String,
+    outputSchema: StructType,
+    options: CaseInsensitiveStringMap
+  ) extends MicroBatchStream with Logging {
+  private def createDataSourceFunc =
+    ds.source.createPythonFunction(
+      ds.getOrCreateDataSourceInPython(shortName, options, Some(outputSchema)).dataSource)
+
+  val runner: PythonStreamingSourceRunner =
+    new PythonStreamingSourceRunner(createDataSourceFunc, outputSchema)
+  runner.init()
+
+  override def initialOffset(): Offset = PythonStreamingSourceOffset(runner.initialOffset())
+
+  override def latestOffset(): Offset = PythonStreamingSourceOffset(runner.latestOffset())
+
+
+  override def planInputPartitions(start: Offset, end: Offset): Array[InputPartition] = {
+    runner.partitions(start.asInstanceOf[PythonStreamingSourceOffset].json,
+      end.asInstanceOf[PythonStreamingSourceOffset].json).map(PythonStreamingSourcePartition(_))
+  }
+
+  override def createReaderFactory(): PartitionReaderFactory = {
+    // TODO: fill in the implementation.

Review Comment:
   Let's file a JIRA ticket and put the ticket number.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(

Review Comment:
   All protected fields in this class: do they have to be protected? Change to private unless needed.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    // Send configurations
+    dataOut.writeInt(SQLConf.get.arrowMaxRecordsPerBatch)
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(latestOffsetFuncId)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(initialOffsetFuncId)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(

Review Comment:
   ditto



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"

Review Comment:
   Could you please add some guide comments for reviewers e.g. if you copied a block of code, where you copied the code from? It would greatly help to avoid looking into all details for "existing and already reviewed" code.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    // Send configurations
+    dataOut.writeInt(SQLConf.get.arrowMaxRecordsPerBatch)
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(latestOffsetFuncId)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(initialOffsetFuncId)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(partitionsFuncId)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(

Review Comment:
   ditto



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    // Send configurations
+    dataOut.writeInt(SQLConf.get.arrowMaxRecordsPerBatch)
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(latestOffsetFuncId)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(initialOffsetFuncId)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(partitionsFuncId)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "planPartitions", msg)
+    }
+    for (_ <- 0 until numPartitions) {
+      val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn)
+      pickledPartitions.append(pickledPartition)
+    }
+    pickledPartitions.toArray
+  }
+
+  /**
+   * Invokes commit(end) function of the stream reader and receive the return value.
+   */
+  def commit(end: String): Unit = {
+    dataOut.writeInt(commitFuncId)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    val status = dataIn.readInt()
+    if (status == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.execution.python
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs}
+import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase {
+
+  protected def simpleDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class SimpleDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        return {"0": "2"}
+      |    def latestOffset(self):
+      |        return {"0": "2"}

Review Comment:
   why not use different offset to distinguish?



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1482217627


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val partitionsFuncId = 886
+  val latestOffsetsFuncId = 887
+}
+
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for streaming functions. Sets up Spark Connect session

Review Comment:
   sorry, the code was copied from else where and I forgot to change this line of comment, fixed.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +305,90 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):

Review Comment:
   Should have versionadded directives, with some docstring



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +305,90 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    def initial_offset(self) -> dict:
+        """
+        The initial offset of the streaming data source.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   Should have versionadded directives.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1491464370


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.execution.python
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs}
+import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase {
+
+  protected def simpleDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class SimpleDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        return {"offset": "0"}
+      |    def latestOffset(self):
+      |        return {"offset": "2"}
+      |    def partitions(self, start: dict, end: dict):
+      |        return [InputPartition(i) for i in range(int(start["offset"]))]
+      |    def commit(self, end: dict):
+      |        1 + 2
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  protected def errorDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class ErrorDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        raise Exception("error reading initial offset")
+      |    def latestOffset(self):
+      |        raise Exception("error reading latest offset")
+      |    def partitions(self, start: dict, end: dict):
+      |        raise Exception("error planning partitions")
+      |    def commit(self, end: dict):
+      |        raise Exception("error committing offset")
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  private val errorDataSourceName = "ErrorDataSource"
+
+  test("simple data stream source") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |$simpleDataStreamReaderScript
+         |
+         |class $dataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        return SimpleDataStreamReader()
+         |""".stripMargin
+    val inputSchema = StructType.fromDDL("input BINARY")
+
+    val dataSource = createUserDefinedPythonDataSource(dataSourceName, dataSourceScript)
+    spark.dataSource.registerPython(dataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("SimpleDataSource")
+    val stream = new PythonMicroBatchStream(
+      pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+
+    val initialOffset = stream.initialOffset()
+    assert(initialOffset.json == "{\"offset\": \"0\"}")
+    for (_ <- 1 to 50) {
+      val offset = stream.latestOffset()
+      assert(offset.json == "{\"offset\": \"2\"}")
+      assert(stream.planInputPartitions(offset, offset).size == 2)
+      stream.commit(offset)
+    }
+    stream.stop()

Review Comment:
   read will be implemented in the next PR, it is executed in executor.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,178 @@
+#
+# 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.
+#
+
+import os
+import sys
+import functools
+import json
+from itertools import islice
+from typing import IO, List, Iterator, Iterable
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.connect.conversion import ArrowTableToRowsConversion, LocalDataToArrowConversion
+from pyspark.sql.datasource import DataSource, InputPartition
+from pyspark.sql.pandas.types import to_arrow_schema
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    BinaryType,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader, outfile):
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader, outfile):
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader, infile, outfile):
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader, infile, outfile):
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)

Review Comment:
   What is the expected behavior if user does not specify the schema and relies on the schema of data source, which we say, inference (which is mostly applied to the majority of data sources)?



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:

Review Comment:
   python will always fail in runtime, so it's more likely whichever error message is better to understand...
   
   But, I don't know, IDE could be the rescue. @HyukjinKwon Will IDE be able to point out if we define this as abstract and implementation class does not implement this?



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:

Review Comment:
   IDE seems not catching this case. Things are runtime as Jungtaek said so it'd be difficult to statically prevent.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.

Review Comment:
   Say `str, int, bool`, as these are actual representation of "python" type? 
   https://docs.python.org/3/library/json.html#json.dump
   
   Also do we disallow other types or they are just discouraged to use (for us) so we don't document? There are more types JSON module in the python standard library supports.
   https://docs.python.org/3/library/json.html#json.JSONDecoder



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def initialOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.execution.python
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs}
+import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase {
+
+  protected def simpleDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class SimpleDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        return {"offset": {"partition-1": 0}}
+      |    def latestOffset(self):
+      |        return {"offset": {"partition-1": 2}}
+      |    def partitions(self, start: dict, end: dict):
+      |        start_index = start["offset"]["partition-1"]
+      |        end_index = end["offset"]["partition-1"]
+      |        return [InputPartition(i) for i in range(start_index, end_index)]
+      |    def commit(self, end: dict):
+      |        1 + 2
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  protected def errorDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class ErrorDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        raise Exception("error reading initial offset")
+      |    def latestOffset(self):
+      |        raise Exception("error reading latest offset")
+      |    def partitions(self, start: dict, end: dict):
+      |        raise Exception("error planning partitions")
+      |    def commit(self, end: dict):
+      |        raise Exception("error committing offset")
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  private val errorDataSourceName = "ErrorDataSource"
+
+  test("simple data stream source") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |$simpleDataStreamReaderScript
+         |
+         |class $dataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        return SimpleDataStreamReader()
+         |""".stripMargin
+    val inputSchema = StructType.fromDDL("input BINARY")
+
+    val dataSource = createUserDefinedPythonDataSource(dataSourceName, dataSourceScript)
+    spark.dataSource.registerPython(dataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("SimpleDataSource")
+    val stream = new PythonMicroBatchStream(
+      pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+
+    val initialOffset = stream.initialOffset()
+    assert(initialOffset.json == "{\"offset\": {\"partition-1\": 0}}")
+    for (_ <- 1 to 50) {

Review Comment:
   Is there any meaning of doing this? It's just doing the same for 50 times.



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def initialOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def latestOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod

Review Comment:
   I guess the above call applies here - if the above methods are expected to be implemented, why we don't use the same error logic here?



##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,160 @@
+#
+# 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.
+#
+
+import os
+import sys
+import json
+from typing import IO
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.datasource import DataSource, DataSourceStreamReader
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+INITIAL_OFFSET_FUNC_ID = 884
+LATEST_OFFSET_FUNC_ID = 885
+PARTITIONS_FUNC_ID = 886
+COMMIT_FUNC_ID = 887
+
+
+def initial_offset_func(reader: DataSourceStreamReader, outfile: IO) -> None:
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader: DataSourceStreamReader, outfile: IO) -> None:
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader: DataSourceStreamReader, infile: IO, outfile: IO) -> None:
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader: DataSourceStreamReader, infile: IO, outfile: IO) -> None:
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)
+        schema = _parse_datatype_json_string(schema_json)
+        if not isinstance(schema, StructType):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "an output schema of type 'StructType'",
+                    "actual": f"'{type(schema).__name__}'",
+                },
+            )
+
+        # Instantiate data source reader.
+        try:
+            reader = data_source.streamReader(schema=schema)
+            # Initialization succeed.
+            write_int(0, outfile)
+            outfile.flush()
+
+            # handle method call from socket
+            while True:
+                func_id = read_int(infile)
+                if func_id == INITIAL_OFFSET_FUNC_ID:
+                    initial_offset_func(reader, outfile)
+                elif func_id == LATEST_OFFSET_FUNC_ID:
+                    latest_offset_func(reader, outfile)
+                elif func_id == PARTITIONS_FUNC_ID:
+                    partitions_func(reader, infile, outfile)
+                elif func_id == COMMIT_FUNC_ID:
+                    commit_func(reader, infile, outfile)
+                outfile.flush()

Review Comment:
   Should we explicitly throw exception if func_id is unknown? Should help fail-fast if something is out of sync.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala:
##########
@@ -28,13 +28,9 @@ import org.apache.spark.sql.test.SharedSparkSession
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.util.Utils
 
-class PythonDataSourceSuite extends QueryTest with SharedSparkSession {
-  import IntegratedUDFTestUtils._
-
-  setupTestData()
+class PythonDataSourceSuiteBase extends QueryTest with SharedSparkSession {

Review Comment:
   If we expect this class to be not instantiated, let's make it be explicit, abstract class.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +305,90 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    def initial_offset(self) -> dict:
+        """
+        The initial offset of the streaming data source.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   Should have versionadded directives.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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

   The implementation seems aligned to the batch side so structure seems fine.


-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1479207218


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +305,90 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    def initial_offset(self) -> dict:

Review Comment:
   Changed to camel case



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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

   Given we have a reference PR now, shall we think through better breakdown of PRs? Some of individual PR may not need to have a test (e.g. interface definition) if you are fully ensuring that you have enough test coverage in reference PR.
   
   My suggestion would be following: at least interface must be separated out. We should have an extensive review from the interface PR, especially for interface/method contract. After that, you can revisit the remaining and see how you can help reviewers to review with smaller but incremental changes.


-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1517356903


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  // When the python process for python_streaming_source_runner receives one of the
+  // integers below, it will invoke the corresponding function of StreamReader instance.
+  val INITIAL_OFFSET_FUNC_ID = 884
+  val LATEST_OFFSET_FUNC_ID = 885
+  val PARTITIONS_FUNC_ID = 886
+  val COMMIT_FUNC_ID = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  private val conf = SparkEnv.get.conf
+  private val bufferSize: Int = conf.get(BUFFER_SIZE)
+  private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  private val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(LATEST_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(INITIAL_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(PARTITIONS_FUNC_ID)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "planPartitions", msg)
+    }
+    for (_ <- 0 until numPartitions) {
+      val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn)
+      pickledPartitions.append(pickledPartition)
+    }
+    pickledPartitions.toArray
+  }
+
+  /**
+   * Invokes commit(end) function of the stream reader and receive the return value.
+   */
+  def commit(end: String): Unit = {
+    dataOut.writeInt(COMMIT_FUNC_ID)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    val status = dataIn.readInt()
+    if (status == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "commitSource", msg)
+    }
+  }
+
+  /**
+   * Stop the python worker process and invoke stop() on stream reader.
+   */
+  def stop(): Unit = {
+    logInfo(s"Stopping streaming runner for module: $workerModule.")
+    try {
+      pythonWorkerFactory.foreach { factory =>
+        pythonWorker.foreach { worker =>
+          factory.stopWorker(worker)
+          factory.stop()

Review Comment:
   stop() is called by finally of the python worker.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def initialOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def latestOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod
+    def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:

Review Comment:
   It has to be `Iterator[Tuple]`. We should fix batch side too. Row inherits `tuple` Mind fixing it here together if you don't mind. cc @allisonwang-db 



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1490111896


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   That essentially means we need to add the deserializeOffset() interface to the StreamReader for deserialization. Or is there any alternative?



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1490106637


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   Does that mean we also need to add deserializeOffset() to the python interface?



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   UPDATE: we are discussing offline now and it could take time - this is tied to the interface and crazily uneasy to change after it is shipped.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1490215001


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.execution.python
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs}
+import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase {
+
+  protected def simpleDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class SimpleDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        return {"0": "2"}
+      |    def latestOffset(self):
+      |        return {"0": "2"}

Review Comment:
   Nice suggestion, changed.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    // Send configurations
+    dataOut.writeInt(SQLConf.get.arrowMaxRecordsPerBatch)
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(latestOffsetFuncId)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(

Review Comment:
   If we have shortName then why not. Ideally we'd be better to generate UUID per source and use it to track for each source, but shortName would be fine for many cases, so.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  // When the python process for python_streaming_source_runner receives one of the
+  // integers below, it will invoke the corresponding function of StreamReader instance.
+  val INITIAL_OFFSET_FUNC_ID = 884
+  val LATEST_OFFSET_FUNC_ID = 885
+  val PARTITIONS_FUNC_ID = 886
+  val COMMIT_FUNC_ID = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  private val conf = SparkEnv.get.conf
+  private val bufferSize: Int = conf.get(BUFFER_SIZE)
+  private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  private val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(LATEST_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(INITIAL_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(PARTITIONS_FUNC_ID)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "planPartitions", msg)
+    }
+    for (_ <- 0 until numPartitions) {
+      val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn)
+      pickledPartitions.append(pickledPartition)
+    }
+    pickledPartitions.toArray
+  }
+
+  /**
+   * Invokes commit(end) function of the stream reader and receive the return value.
+   */
+  def commit(end: String): Unit = {
+    dataOut.writeInt(COMMIT_FUNC_ID)

Review Comment:
   I'm wondering if the python process somehow dies, would this throw an error? 
   
   In UDF runners I believe the job just aborted and restarted so should be fine. But on driver, would this be an issue? Should we generally do try-catch here or somewhere in upstream logic? cc @chaoqin-li1123 @HyukjinKwon @allisonwang-db 



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)

Review Comment:
   Yes I agree we don't need to do this temporary set-reset here. This logic was in connect StreamingPythonRunner because of some concurrnet development issue. I'll also create a PR to remove it 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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1517359698


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  // When the python process for python_streaming_source_runner receives one of the
+  // integers below, it will invoke the corresponding function of StreamReader instance.
+  val INITIAL_OFFSET_FUNC_ID = 884
+  val LATEST_OFFSET_FUNC_ID = 885
+  val PARTITIONS_FUNC_ID = 886
+  val COMMIT_FUNC_ID = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  private val conf = SparkEnv.get.conf
+  private val bufferSize: Int = conf.get(BUFFER_SIZE)
+  private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  private val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(LATEST_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(INITIAL_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(PARTITIONS_FUNC_ID)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "planPartitions", msg)
+    }
+    for (_ <- 0 until numPartitions) {
+      val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn)
+      pickledPartitions.append(pickledPartition)
+    }
+    pickledPartitions.toArray
+  }
+
+  /**
+   * Invokes commit(end) function of the stream reader and receive the return value.
+   */
+  def commit(end: String): Unit = {
+    dataOut.writeInt(COMMIT_FUNC_ID)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    val status = dataIn.readInt()
+    if (status == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "commitSource", msg)
+    }
+  }
+
+  /**
+   * Stop the python worker process and invoke stop() on stream reader.
+   */
+  def stop(): Unit = {
+    logInfo(s"Stopping streaming runner for module: $workerModule.")
+    try {
+      pythonWorkerFactory.foreach { factory =>
+        pythonWorker.foreach { worker =>
+          factory.stopWorker(worker)
+          factory.stop()
+        }
+      }
+    } catch {
+      case e: Exception =>
+        logError("Exception when trying to kill worker", e)
+    }

Review Comment:
   I feel log error should be sufficient. Failing to close python worker may be a system error instead of data source failure.



-- 
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


Re: [PR] [SPARK-46962[SS] Implement python worker to run python streaming data source [spark]

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

   unsubscribe
   Thanks,
   Gabriel
   
   
   
   On Mon, Feb 5, 2024 at 5:25 AM chaoqin-li1123 ***@***.***>
   wrote:
   
   > What changes were proposed in this pull request?
   >
   > Implement python worker to run python streaming data source and
   > communicate with JVM through socket. Create a PythonMicrobatchStream to
   > invoke RPC function call.
   > Why are the changes needed?
   >
   > In preparation for support of development of streaming data source in
   > Python.
   > Does this PR introduce *any* user-facing change?
   >
   > No.
   > How was this patch tested?
   >
   > Unit test.
   > Was this patch authored or co-authored using generative AI tooling?
   >
   > No.
   > ------------------------------
   > You can view, comment on, or merge this pull request online at:
   >
   >   https://github.com/apache/spark/pull/45023
   > Commit Summary
   >
   >    - 475b522
   >    <https://github.com/apache/spark/pull/45023/commits/475b5221315882220fb90a756049b03cbac642da>
   >    implementation
   >    - 2f8f846
   >    <https://github.com/apache/spark/pull/45023/commits/2f8f8467eb6047b8a0a52638b878a2e049b5f643>
   >    clean up
   >    - b39e3be
   >    <https://github.com/apache/spark/pull/45023/commits/b39e3becc1768b913a2e3709e760a98935a0ca68>
   >    clean up
   >    - b355c96
   >    <https://github.com/apache/spark/pull/45023/commits/b355c96a58db395600e332808883d3085f623c46>
   >    clean up
   >
   > File Changes
   >
   > (8 files <https://github.com/apache/spark/pull/45023/files>)
   >
   >    - *A* python/__init__.py
   >    <https://github.com/apache/spark/pull/45023/files#diff-6782036b11fde2c417fa4d886777898de32ff7edff2d3d0e6171614516127f3a>
   >    (0)
   >    - *M* python/pyspark/sql/datasource.py
   >    <https://github.com/apache/spark/pull/45023/files#diff-1311df23a3097eed5cf3c629e4a3f53c2656053bb18e89d40c5c108c2d1ff1cb>
   >    (91)
   >    - *A* python/pyspark/sql/streaming/python_streaming_source_runner.py
   >    <https://github.com/apache/spark/pull/45023/files#diff-772778ddbeb0c9da6b17ec861201b64556f8920d49aec5e145ee0f004411bf23>
   >    (176)
   >    - *M*
   >    sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonScan.scala
   >    <https://github.com/apache/spark/pull/45023/files#diff-8735bc8c9030aef89ff35f8fdfe8de545bbddd25647d57965245f7c0bb5c28cf>
   >    (68)
   >    - *M*
   >    sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala
   >    <https://github.com/apache/spark/pull/45023/files#diff-c762c3fb19181774c271ce6527938553e3fc4f1b107b95c7f7a7728e1371def3>
   >    (2)
   >    - *A*
   >    sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala
   >    <https://github.com/apache/spark/pull/45023/files#diff-41b5c2de33892248fbb71d7f8034e2e94c35dd243c38dd90a5a216730a42dd8f>
   >    (167)
   >    - *M*
   >    sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala
   >    <https://github.com/apache/spark/pull/45023/files#diff-419822ba55838aed0d6439beff1ba0a87041c32724972d4036c0eda096e66b0b>
   >    (36)
   >    - *A*
   >    sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala
   >    <https://github.com/apache/spark/pull/45023/files#diff-aac0c53cf10992b58e7862115c9f72b1cb5b086a39b4e5b11543ac99d234f761>
   >    (131)
   >
   > Patch Links:
   >
   >    - https://github.com/apache/spark/pull/45023.patch
   >    - https://github.com/apache/spark/pull/45023.diff
   >
   > —
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/spark/pull/45023>, or unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/ABQLF6XZPZMZR7WUBDJXOT3YR74FLAVCNFSM6AAAAABCZE5EO6VHI2DSMVQWIX3LMV43ASLTON2WKOZSGEYTOMZRGU2DANY>
   > .
   > You are receiving this because you are subscribed to this thread.Message
   > ID: ***@***.***>
   >
   


-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1520455477


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  // When the python process for python_streaming_source_runner receives one of the
+  // integers below, it will invoke the corresponding function of StreamReader instance.
+  val INITIAL_OFFSET_FUNC_ID = 884
+  val LATEST_OFFSET_FUNC_ID = 885
+  val PARTITIONS_FUNC_ID = 886
+  val COMMIT_FUNC_ID = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  private val conf = SparkEnv.get.conf
+  private val bufferSize: Int = conf.get(BUFFER_SIZE)
+  private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  private val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(LATEST_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(INITIAL_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(PARTITIONS_FUNC_ID)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "planPartitions", msg)
+    }
+    for (_ <- 0 until numPartitions) {
+      val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn)
+      pickledPartitions.append(pickledPartition)
+    }
+    pickledPartitions.toArray
+  }
+
+  /**
+   * Invokes commit(end) function of the stream reader and receive the return value.
+   */
+  def commit(end: String): Unit = {
+    dataOut.writeInt(COMMIT_FUNC_ID)

Review Comment:
   The `PythonPlannerRunner` used in batch implementation catches errors from the Python process:
   ```
       val length = dataIn.readInt()
       if (length == SpecialLengths.PYTHON_EXCEPTION_THROWN) { // throw exceptions
   ```
   But I am not sure what the behavior is if the Python process dies. cc @ueshin 



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1490256089


##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,178 @@
+#
+# 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.
+#
+
+import os
+import sys
+import functools
+import json
+from itertools import islice
+from typing import IO, List, Iterator, Iterable
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.connect.conversion import ArrowTableToRowsConversion, LocalDataToArrowConversion
+from pyspark.sql.datasource import DataSource, InputPartition
+from pyspark.sql.pandas.types import to_arrow_schema
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    BinaryType,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader, outfile):
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader, outfile):
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader, infile, outfile):
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader, infile, outfile):
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)

Review Comment:
   We only use data source schema when user doesn't specify schema.
   https://github.com/apache/spark/blob/736d8ab3f00e7c5ba1b01c22f6398b636b8492ea/python/pyspark/sql/worker/create_data_source.py#L144



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -426,6 +426,10 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
             in the final DataFrame.
         """
         ...
+        raise PySparkNotImplementedError(

Review Comment:
   @chaoqin-li1123 
   nit: if not needed, let's remove @abstract in the method definition.
   Please do this as follow-up PR. I'd like to unblock this PR to let you move forward.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {

Review Comment:
   Can we please document the protocol for this runner <-> driver? This is a new worker specialized to Python streaming source - let's not force code readers to figure out by themselves.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1482218173


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {

Review Comment:
   Comments added.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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

   As suggested by @HeartSaVioR , I bring up a reference PR that contains the full change and end2end example of python streaming data source https://github.com/apache/spark/pull/45065/files#diff-aac0c53cf10992b58e7862115c9f72b1cb5b086a39b4e5b11543ac99d234f761R122


-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1490214280


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    // Send configurations
+    dataOut.writeInt(SQLConf.get.arrowMaxRecordsPerBatch)
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(latestOffsetFuncId)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(

Review Comment:
   How do we identify source, do you think we should attach the shornName of the source in the error message?



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1490211705


##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,178 @@
+#
+# 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.
+#
+
+import os
+import sys
+import functools
+import json
+from itertools import islice
+from typing import IO, List, Iterator, Iterable
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.connect.conversion import ArrowTableToRowsConversion, LocalDataToArrowConversion
+from pyspark.sql.datasource import DataSource, InputPartition
+from pyspark.sql.pandas.types import to_arrow_schema
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    BinaryType,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader, outfile):
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader, outfile):
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader, infile, outfile):
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader, infile, outfile):
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)
+        schema = _parse_datatype_json_string(schema_json)
+        if not isinstance(schema, StructType):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "an output schema of type 'StructType'",
+                    "actual": f"'{type(schema).__name__}'",
+                },
+            )
+
+        # Receive the configuration values.
+        max_arrow_batch_size = read_int(infile)
+        assert max_arrow_batch_size > 0, (
+            "The maximum arrow batch size should be greater than 0, but got "
+            f"'{max_arrow_batch_size}'"
+        )
+
+        # Instantiate data source reader.
+        try:
+            reader = data_source.streamReader(schema=schema)
+            # Initialization succeed.
+            write_int(0, outfile)
+            outfile.flush()
+
+            # handle method call from socket
+            while True:
+                func_id = read_int(infile)
+                if func_id == initial_offset_func_id:
+                    initial_offset_func(reader, outfile)
+                elif func_id == latest_offset_func_id:
+                    latest_offset_func(reader, outfile)
+                elif func_id == partitions_func_id:
+                    partitions_func(reader, infile, outfile)
+                elif func_id == commit_func_id:
+                    commit_func(reader, infile, outfile)
+                outfile.flush()
+
+        except NotImplementedError:
+            raise PySparkRuntimeError(
+                error_class="PYTHON_DATA_SOURCE_METHOD_NOT_IMPLEMENTED",
+                message_parameters={"type": "reader", "method": "reader"},
+            )
+        except Exception as e:

Review Comment:
   Changed to PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR error class.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1490213270


##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,178 @@
+#
+# 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.
+#
+
+import os
+import sys
+import functools
+import json
+from itertools import islice
+from typing import IO, List, Iterator, Iterable
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.connect.conversion import ArrowTableToRowsConversion, LocalDataToArrowConversion
+from pyspark.sql.datasource import DataSource, InputPartition
+from pyspark.sql.pandas.types import to_arrow_schema
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    BinaryType,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader, outfile):
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader, outfile):
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader, infile, outfile):
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader, infile, outfile):
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)
+        schema = _parse_datatype_json_string(schema_json)
+        if not isinstance(schema, StructType):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "an output schema of type 'StructType'",
+                    "actual": f"'{type(schema).__name__}'",
+                },
+            )
+
+        # Receive the configuration values.

Review Comment:
   Removed. It is no longer necessary because the pickled read function will be created from another python worker.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1490557068


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict

Review Comment:
   Nit: Dict[str, str]? Is this case sensitive or case insensitive?



##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+#
+
+import os
+import sys
+import json
+from typing import IO
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.datasource import DataSource
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader, outfile):
+    offset = reader.initialOffset()

Review Comment:
   What if the initialOffset is not implemented?



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns

Review Comment:
   ditto for examples



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+

Review Comment:
   Could you also provide an example of what the dictionary looks like?
   ```
   Examples
   ---------
   ...
   ```



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod
+    def read(self, partition) -> Iterator[Union[Tuple, Row]]:
+        """
+        Generates data for a given partition and returns an iterator of tuples or rows.
+
+        This method is invoked once per partition to read the data. Implementing
+        this method is required for stream reader. You can initialize any
+        non-serializable resources required for reading data from the data source
+        within this method.
+        This method is static and stateless. You shouldn't access mutable class member
+        or keep in memory state between different invocations of read().
+
+        Parameters
+        ----------
+        partition : object

Review Comment:
   ```suggestion
           partition : InputPartition
   ```



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.

Review Comment:
   Is it better to have another class named `Offset` instead of using a dictionary type here?



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod
+    def read(self, partition) -> Iterator[Union[Tuple, Row]]:
+        """
+        Generates data for a given partition and returns an iterator of tuples or rows.
+
+        This method is invoked once per partition to read the data. Implementing
+        this method is required for stream reader. You can initialize any
+        non-serializable resources required for reading data from the data source
+        within this method.
+        This method is static and stateless. You shouldn't access mutable class member
+        or keep in memory state between different invocations of read().
+
+        Parameters
+        ----------
+        partition : object
+            The partition to read. It must be one of the partition values returned by
+            ``partitions()``.
+
+        Returns
+        -------
+        Iterator[Tuple] or Iterator[Row]
+            An iterator of tuples or rows. Each tuple or row will be converted to a row
+            in the final DataFrame.
+        """
+        ...
+
+    def commit(self, end: dict):
+        """
+        Informs the source that Spark has completed processing all data for offsets less than or
+        equal to `end` and will only request offsets greater than `end` in the future.
+
+        Parameters
+        ----------
+        end : dict
+            The latest offset that the streaming query has processed for this source.
+        """
+        ...
+
+    def stop(self):

Review Comment:
   ditto for return type



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod
+    def read(self, partition) -> Iterator[Union[Tuple, Row]]:
+        """
+        Generates data for a given partition and returns an iterator of tuples or rows.
+
+        This method is invoked once per partition to read the data. Implementing
+        this method is required for stream reader. You can initialize any
+        non-serializable resources required for reading data from the data source
+        within this method.
+        This method is static and stateless. You shouldn't access mutable class member
+        or keep in memory state between different invocations of read().

Review Comment:
   This part is very important and can be under the Notes section.
   ```
   Notes
   ------
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.execution.python
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs}
+import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase {
+
+  protected def simpleDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class SimpleDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        return {"offset": "0"}
+      |    def latestOffset(self):
+      |        return {"offset": "2"}
+      |    def partitions(self, start: dict, end: dict):
+      |        return [InputPartition(i) for i in range(int(start["offset"]))]
+      |    def commit(self, end: dict):
+      |        1 + 2
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  protected def errorDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class ErrorDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        raise Exception("error reading initial offset")
+      |    def latestOffset(self):
+      |        raise Exception("error reading latest offset")
+      |    def partitions(self, start: dict, end: dict):
+      |        raise Exception("error planning partitions")
+      |    def commit(self, end: dict):
+      |        raise Exception("error committing offset")
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  private val errorDataSourceName = "ErrorDataSource"
+
+  test("simple data stream source") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |$simpleDataStreamReaderScript
+         |
+         |class $dataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        return SimpleDataStreamReader()
+         |""".stripMargin
+    val inputSchema = StructType.fromDDL("input BINARY")
+
+    val dataSource = createUserDefinedPythonDataSource(dataSourceName, dataSourceScript)
+    spark.dataSource.registerPython(dataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("SimpleDataSource")
+    val stream = new PythonMicroBatchStream(
+      pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+
+    val initialOffset = stream.initialOffset()
+    assert(initialOffset.json == "{\"offset\": \"0\"}")
+    for (_ <- 1 to 50) {
+      val offset = stream.latestOffset()
+      assert(offset.json == "{\"offset\": \"2\"}")
+      assert(stream.planInputPartitions(offset, offset).size == 2)
+      stream.commit(offset)
+    }
+    stream.stop()

Review Comment:
   I am not super familiar with streaming, but could you also check the output/result of each stream read? 



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod
+    def read(self, partition) -> Iterator[Union[Tuple, Row]]:
+        """
+        Generates data for a given partition and returns an iterator of tuples or rows.
+
+        This method is invoked once per partition to read the data. Implementing
+        this method is required for stream reader. You can initialize any
+        non-serializable resources required for reading data from the data source
+        within this method.
+        This method is static and stateless. You shouldn't access mutable class member
+        or keep in memory state between different invocations of read().
+
+        Parameters
+        ----------
+        partition : object
+            The partition to read. It must be one of the partition values returned by
+            ``partitions()``.
+
+        Returns
+        -------
+        Iterator[Tuple] or Iterator[Row]
+            An iterator of tuples or rows. Each tuple or row will be converted to a row
+            in the final DataFrame.
+        """
+        ...
+
+    def commit(self, end: dict):

Review Comment:
   Please add return type



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1491463836


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod
+    def read(self, partition) -> Iterator[Union[Tuple, Row]]:
+        """
+        Generates data for a given partition and returns an iterator of tuples or rows.
+
+        This method is invoked once per partition to read the data. Implementing
+        this method is required for stream reader. You can initialize any
+        non-serializable resources required for reading data from the data source
+        within this method.
+        This method is static and stateless. You shouldn't access mutable class member
+        or keep in memory state between different invocations of read().
+
+        Parameters
+        ----------
+        partition : object
+            The partition to read. It must be one of the partition values returned by
+            ``partitions()``.
+
+        Returns
+        -------
+        Iterator[Tuple] or Iterator[Row]
+            An iterator of tuples or rows. Each tuple or row will be converted to a row
+            in the final DataFrame.
+        """
+        ...
+
+    def commit(self, end: dict):

Review Comment:
   This function is not supposed to return anything, how do we express that in python?



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +305,90 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    def initial_offset(self) -> dict:
+        """
+        The initial offset of the streaming data source.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+
+    def latest_offset(self) -> dict:
+        """
+        Seek the latest offset of the streaming data source.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Plan the read partitions given the start and end offset.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]

Review Comment:
   It shouldn't actually use plain python type hints here but a description of the types (see the example in https://numpydoc.readthedocs.io/en/latest/format.html#parameters). It should be something like ``` sequence of :class:`InputPartition` ```



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1501273741


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict

Review Comment:
   Changed to dict or recursive whose key and value must be primitive type.



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+

Review Comment:
   Example added.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR closed pull request #45023: [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source
URL: https://github.com/apache/spark/pull/45023


-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  // When the python process for python_streaming_source_runner receives one of the
+  // integers below, it will invoke the corresponding function of StreamReader instance.
+  val INITIAL_OFFSET_FUNC_ID = 884
+  val LATEST_OFFSET_FUNC_ID = 885
+  val PARTITIONS_FUNC_ID = 886
+  val COMMIT_FUNC_ID = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  private val conf = SparkEnv.get.conf
+  private val bufferSize: Int = conf.get(BUFFER_SIZE)
+  private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  private val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {

Review Comment:
   Could either @rangadi or @WweiL take a look? This is somewhat similar with foreachBatch.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:

Review Comment:
   Should we make this abstract to force user to implement it? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  // When the python process for python_streaming_source_runner receives one of the
+  // integers below, it will invoke the corresponding function of StreamReader instance.
+  val INITIAL_OFFSET_FUNC_ID = 884
+  val LATEST_OFFSET_FUNC_ID = 885
+  val PARTITIONS_FUNC_ID = 886
+  val COMMIT_FUNC_ID = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  private val conf = SparkEnv.get.conf
+  private val bufferSize: Int = conf.get(BUFFER_SIZE)
+  private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  private val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(LATEST_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(INITIAL_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(PARTITIONS_FUNC_ID)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "planPartitions", msg)
+    }
+    for (_ <- 0 until numPartitions) {
+      val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn)
+      pickledPartitions.append(pickledPartition)
+    }
+    pickledPartitions.toArray
+  }
+
+  /**
+   * Invokes commit(end) function of the stream reader and receive the return value.
+   */
+  def commit(end: String): Unit = {
+    dataOut.writeInt(COMMIT_FUNC_ID)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    val status = dataIn.readInt()
+    if (status == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "commitSource", msg)
+    }
+  }
+
+  /**
+   * Stop the python worker process and invoke stop() on stream reader.
+   */
+  def stop(): Unit = {
+    logInfo(s"Stopping streaming runner for module: $workerModule.")
+    try {
+      pythonWorkerFactory.foreach { factory =>
+        pythonWorker.foreach { worker =>
+          factory.stopWorker(worker)
+          factory.stop()
+        }
+      }
+    } catch {
+      case e: Exception =>
+        logError("Exception when trying to kill worker", e)
+    }

Review Comment:
   Is it safe to swallow this?



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def initialOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def latestOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod
+    def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:

Review Comment:
   I am not completely familiar with internals of Unions in Python, however ..
   
   We do not expect values in the Iterator to differ in type, i.e., either the entire output is Iterator[Tuple] or Iterator[Row]. Should the return type be then - `Union[Iterator{Tuple], Iterator[Row]]`? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  // When the python process for python_streaming_source_runner receives one of the
+  // integers below, it will invoke the corresponding function of StreamReader instance.
+  val INITIAL_OFFSET_FUNC_ID = 884
+  val LATEST_OFFSET_FUNC_ID = 885
+  val PARTITIONS_FUNC_ID = 886
+  val COMMIT_FUNC_ID = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  private val conf = SparkEnv.get.conf
+  private val bufferSize: Int = conf.get(BUFFER_SIZE)
+  private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  private val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(LATEST_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(INITIAL_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(PARTITIONS_FUNC_ID)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "planPartitions", msg)
+    }
+    for (_ <- 0 until numPartitions) {
+      val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn)
+      pickledPartitions.append(pickledPartition)
+    }
+    pickledPartitions.toArray
+  }
+
+  /**
+   * Invokes commit(end) function of the stream reader and receive the return value.
+   */
+  def commit(end: String): Unit = {
+    dataOut.writeInt(COMMIT_FUNC_ID)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    val status = dataIn.readInt()
+    if (status == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "commitSource", msg)
+    }
+  }
+
+  /**
+   * Stop the python worker process and invoke stop() on stream reader.
+   */
+  def stop(): Unit = {
+    logInfo(s"Stopping streaming runner for module: $workerModule.")
+    try {
+      pythonWorkerFactory.foreach { factory =>
+        pythonWorker.foreach { worker =>
+          factory.stopWorker(worker)
+          factory.stop()

Review Comment:
   We should have a call to Stop function on datasource here, right?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)

Review Comment:
   I think if a task launches a PythonUDFWorker at this time, we would not use the daemon? @chaoqin-li1123  Rather than setting the config, can we pass a parameter to PythonWorkerFactory to explicitly not use the daemon? That would prevent us from modifying the config. 



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1479091720


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +305,90 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):

Review Comment:
   Added.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1482219079


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonScan.scala:
##########
@@ -45,13 +63,50 @@ class PythonScan(
     new PythonPartitionReaderFactory(
       ds.source, readerFunc, outputSchema, jobArtifactUUID)
   }
+}
 
-  override def toBatch: Batch = this
+case class PythonStreamingSourceOffset(json: String) extends Offset
 
-  override def description: String = "(Python)"
+case class PythonStreamingSourcePartition(partition: Array[Byte]) extends InputPartition
 
-  override def readSchema(): StructType = outputSchema
+class PythonMicroBatchStream(

Review Comment:
   Moved to a separate file.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +305,90 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    def initial_offset(self) -> dict:

Review Comment:
   I think the naming should be `initialOffset`.
   
   Or ... you could just have one method called `offset`, and returns a tuple or a Data Class that has `initial` and `latest` property to make this more Pythonic.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1482258013


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,97 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):

Review Comment:
   Great suggestion, I have copied corresponding method comments from scala API.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1492090151


##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,160 @@
+#
+# 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.
+#
+
+import os
+import sys
+import json
+from typing import IO
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.datasource import DataSource, DataSourceStreamReader
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader: DataSourceStreamReader, outfile: IO) -> None:
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader: DataSourceStreamReader, outfile: IO) -> None:
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader: DataSourceStreamReader, infile: IO, outfile: IO) -> None:
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader: DataSourceStreamReader, infile: IO, outfile: IO) -> None:
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)
+        schema = _parse_datatype_json_string(schema_json)
+        if not isinstance(schema, StructType):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "an output schema of type 'StructType'",
+                    "actual": f"'{type(schema).__name__}'",
+                },
+            )
+
+        # Instantiate data source reader.
+        try:
+            reader = data_source.streamReader(schema=schema)
+            # Initialization succeed.
+            write_int(0, outfile)
+            outfile.flush()
+
+            # handle method call from socket
+            while True:
+                func_id = read_int(infile)
+                if func_id == initial_offset_func_id:
+                    initial_offset_func(reader, outfile)
+                elif func_id == latest_offset_func_id:
+                    latest_offset_func(reader, outfile)
+                elif func_id == partitions_func_id:
+                    partitions_func(reader, infile, outfile)
+                elif func_id == commit_func_id:
+                    commit_func(reader, infile, outfile)
+                outfile.flush()
+        except Exception as e:
+            error_msg = 'data source {} throw exception: {}'.format(data_source.name, e)
+            raise PySparkRuntimeError(
+                error_class="PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR",
+                message_parameters={"type": "reader", "error": error_msg},

Review Comment:
   Could you add a test case for this? I don't think the PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR has a parameter called `type`.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884

Review Comment:
   Could you also add some comments on what are these IDs? Also, use
   ```suggestion
     val INITIAL_OFFSET_FUNC_ID = 884
   ```



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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

   Thanks! Merging to master.


-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1516609093


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)

Review Comment:
   Do we need to set this config? `workerFactory.createSimpleWorker(blockingMode = true)` this is already not using the daemon mode.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1517356903


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  // When the python process for python_streaming_source_runner receives one of the
+  // integers below, it will invoke the corresponding function of StreamReader instance.
+  val INITIAL_OFFSET_FUNC_ID = 884
+  val LATEST_OFFSET_FUNC_ID = 885
+  val PARTITIONS_FUNC_ID = 886
+  val COMMIT_FUNC_ID = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  private val conf = SparkEnv.get.conf
+  private val bufferSize: Int = conf.get(BUFFER_SIZE)
+  private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  private val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(LATEST_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(INITIAL_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(PARTITIONS_FUNC_ID)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "planPartitions", msg)
+    }
+    for (_ <- 0 until numPartitions) {
+      val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn)
+      pickledPartitions.append(pickledPartition)
+    }
+    pickledPartitions.toArray
+  }
+
+  /**
+   * Invokes commit(end) function of the stream reader and receive the return value.
+   */
+  def commit(end: String): Unit = {
+    dataOut.writeInt(COMMIT_FUNC_ID)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    val status = dataIn.readInt()
+    if (status == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "commitSource", msg)
+    }
+  }
+
+  /**
+   * Stop the python worker process and invoke stop() on stream reader.
+   */
+  def stop(): Unit = {
+    logInfo(s"Stopping streaming runner for module: $workerModule.")
+    try {
+      pythonWorkerFactory.foreach { factory =>
+        pythonWorker.foreach { worker =>
+          factory.stopWorker(worker)
+          factory.stop()

Review Comment:
   stop() is called by finally block of the python worker.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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

   @HeartSaVioR @HyukjinKwon @allisonwang-db Initial implementation of python streaming data source, PTAL, thanks!


-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1479207581


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +305,90 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    def initial_offset(self) -> dict:
+        """
+        The initial offset of the streaming data source.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+
+    def latest_offset(self) -> dict:
+        """
+        Seek the latest offset of the streaming data source.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Plan the read partitions given the start and end offset.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]

Review Comment:
   Thanks, I feel it is probably cleaner to make separate change in another PR.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,97 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):

Review Comment:
   Small hint: majority of methods we are introducing in Python streaming data source have corresponding methods in Scala API. You are OK to copy the same content, with modification to reflect the diff on contract if any.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1489903039


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   The serialization is easy, but deserialization will be tricky. We don't have runtime type information to deserialize a plain json text back to python object.(because we don't even know what python type the json is serialized from unless we keep extra type information in the json)



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   Maybe yes. I'm OK with dict with clearly explaining which type they can use as value (flipping the coin we need to still restrict the types as we have to convert this to json), but probably less headache to let data source implementation to take care of themselves.



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   Maybe yes. I'm OK with dict with clearly explaining which type they can use as value (flipping the coin we need to still restrict the available types as we have to convert this to json), but probably less headache to let data source implementation to take care of themselves.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   So data source implementation provides serde between its offset model <-> json, and python worker will handle it with serde being provided. For python worker <-> JVM, json is used. Does this make sense?



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1490214832


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.execution.python
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs}
+import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase {
+
+  protected def simpleDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class SimpleDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        return {"0": "2"}
+      |    def latestOffset(self):
+      |        return {"0": "2"}
+      |    def partitions(self, start: dict, end: dict):
+      |        return [InputPartition(i) for i in range(int(start["0"]))]
+      |    def commit(self, end: dict):
+      |        1 + 2
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  protected def errorDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class ErrorDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        raise Exception("error reading initial offset")
+      |    def latestOffset(self):
+      |        raise Exception("error reading latest offset")
+      |    def partitions(self, start: dict, end: dict):
+      |        raise Exception("error planning partitions")
+      |    def commit(self, end: dict):
+      |        raise Exception("error committing offset")
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  private val errorDataSourceName = "ErrorDataSource"
+
+  test("simple data stream source") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |$simpleDataStreamReaderScript
+         |
+         |class $dataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        return SimpleDataStreamReader()
+         |""".stripMargin
+    val inputSchema = StructType.fromDDL("input BINARY")
+
+    val dataSource = createUserDefinedPythonDataSource(dataSourceName, dataSourceScript)
+    spark.dataSource.registerPython(dataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("SimpleDataSource")
+    val stream = new PythonMicroBatchStream(
+      pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+
+    val initialOffset = stream.initialOffset()
+    assert(initialOffset.json == "{\"0\": \"2\"}")
+    for (_ <- 1 to 50) {
+      val offset = stream.latestOffset()
+      assert(offset.json == "{\"0\": \"2\"}")
+      assert(stream.planInputPartitions(offset, offset).size == 2)
+      stream.commit(offset)
+    }
+    stream.stop()
+  }
+
+  test("Error creating stream reader") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |class $dataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        raise Exception("error creating stream reader")
+         |""".stripMargin
+    val dataSource = createUserDefinedPythonDataSource(
+      name = dataSourceName, pythonScript = dataSourceScript)
+    spark.dataSource.registerPython(dataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("SimpleDataSource")
+    val inputSchema = StructType.fromDDL("input BINARY")
+    val err = intercept[AnalysisException] {
+      new PythonMicroBatchStream(
+        pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+    }
+    assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR")
+    assert(err.getMessage.contains("error creating stream reader"))
+  }
+
+  test("Error in stream reader") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |$errorDataStreamReaderScript
+         |
+         |class $errorDataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        return ErrorDataStreamReader()
+         |""".stripMargin
+    val inputSchema = StructType.fromDDL("input BINARY")
+
+    val dataSource = createUserDefinedPythonDataSource(errorDataSourceName, dataSourceScript)
+    spark.dataSource.registerPython(errorDataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("ErrorDataSource")
+    val offset = PythonStreamingSourceOffset("{\"0\": \"2\"}")
+
+    def testMicroBatchStreamError(msg: String)
+                                 (func: PythonMicroBatchStream => Unit): Unit = {
+      val stream = new PythonMicroBatchStream(
+        pythonDs, errorDataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+      val err = intercept[SparkException] {
+        func(stream)
+      }
+      assert(err.getMessage.contains(msg))

Review Comment:
   Changed to checkError.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1518252566


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val initialOffsetFuncId = 884
+  val latestOffsetFuncId = 885
+  val partitionsFuncId = 886
+  val commitFuncId = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  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
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  protected val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)

Review Comment:
   Thanks for pointing it out, I removed the redundant conf reset.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1501274208


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,117 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.

Review Comment:
   We use dict here because we want to enforce the object to be json serializable.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +305,90 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    def initial_offset(self) -> dict:
+        """
+        The initial offset of the streaming data source.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+
+    def latest_offset(self) -> dict:
+        """
+        Seek the latest offset of the streaming data source.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.
+        """
+        ...
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Plan the read partitions given the start and end offset.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]

Review Comment:
   I see there are same mistakes in the batch version so I don't mind fixing it separately for all later.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1482216213


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  val partitionsFuncId = 886
+  val latestOffsetsFuncId = 887

Review Comment:
   They are sent to python process to invoke the corresponding function. I also wonder whether is there better alternative than hardcoding the number in both scala and python code?



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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

   Could you please check the GA build result and fix accordingly?


-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

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


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   Maybe yes. I'm OK with dict with clearly explaining which type they can use as value, but probably less headache to let data source implementation to take care of themselves.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1511914150


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,104 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict whose key and values are str type.

Review Comment:
   We have agreed that the offset interface be dict or recursive dict whose key and value are primitive type offline.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1520368942


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.execution.python
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs}
+import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase {
+
+  protected def simpleDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class SimpleDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        return {"offset": {"partition-1": 0}}
+      |    def latestOffset(self):
+      |        return {"offset": {"partition-1": 2}}
+      |    def partitions(self, start: dict, end: dict):
+      |        start_index = start["offset"]["partition-1"]
+      |        end_index = end["offset"]["partition-1"]
+      |        return [InputPartition(i) for i in range(start_index, end_index)]
+      |    def commit(self, end: dict):
+      |        1 + 2
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  protected def errorDataStreamReaderScript: String =
+    """
+      |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition
+      |
+      |class ErrorDataStreamReader(DataSourceStreamReader):
+      |    def initialOffset(self):
+      |        raise Exception("error reading initial offset")
+      |    def latestOffset(self):
+      |        raise Exception("error reading latest offset")
+      |    def partitions(self, start: dict, end: dict):
+      |        raise Exception("error planning partitions")
+      |    def commit(self, end: dict):
+      |        raise Exception("error committing offset")
+      |    def read(self, partition):
+      |        yield (0, partition.value)
+      |        yield (1, partition.value)
+      |        yield (2, partition.value)
+      |""".stripMargin
+
+  private val errorDataSourceName = "ErrorDataSource"
+
+  test("simple data stream source") {
+    assume(shouldTestPandasUDFs)
+    val dataSourceScript =
+      s"""
+         |from pyspark.sql.datasource import DataSource
+         |$simpleDataStreamReaderScript
+         |
+         |class $dataSourceName(DataSource):
+         |    def streamReader(self, schema):
+         |        return SimpleDataStreamReader()
+         |""".stripMargin
+    val inputSchema = StructType.fromDDL("input BINARY")
+
+    val dataSource = createUserDefinedPythonDataSource(dataSourceName, dataSourceScript)
+    spark.dataSource.registerPython(dataSourceName, dataSource)
+    val pythonDs = new PythonDataSourceV2
+    pythonDs.setShortName("SimpleDataSource")
+    val stream = new PythonMicroBatchStream(
+      pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty())
+
+    val initialOffset = stream.initialOffset()
+    assert(initialOffset.json == "{\"offset\": {\"partition-1\": 0}}")
+    for (_ <- 1 to 50) {

Review Comment:
   It is a stress test to prove that the RPC calls can be invoked many times.



##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,160 @@
+#
+# 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.
+#
+
+import os
+import sys
+import json
+from typing import IO
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.datasource import DataSource, DataSourceStreamReader
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+INITIAL_OFFSET_FUNC_ID = 884
+LATEST_OFFSET_FUNC_ID = 885
+PARTITIONS_FUNC_ID = 886
+COMMIT_FUNC_ID = 887
+
+
+def initial_offset_func(reader: DataSourceStreamReader, outfile: IO) -> None:
+    offset = reader.initialOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def latest_offset_func(reader: DataSourceStreamReader, outfile: IO) -> None:
+    offset = reader.latestOffset()
+    write_with_length(json.dumps(offset).encode("utf-8"), outfile)
+
+
+def partitions_func(reader: DataSourceStreamReader, infile: IO, outfile: IO) -> None:
+    start_offset = json.loads(utf8_deserializer.loads(infile))
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    partitions = reader.partitions(start_offset, end_offset)
+    # Return the serialized partition values.
+    write_int(len(partitions), outfile)
+    for partition in partitions:
+        pickleSer._write_with_length(partition, outfile)
+
+
+def commit_func(reader: DataSourceStreamReader, infile: IO, outfile: IO) -> None:
+    end_offset = json.loads(utf8_deserializer.loads(infile))
+    reader.commit(end_offset)
+    write_int(0, outfile)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    try:
+        check_python_version(infile)
+        setup_spark_files(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+
+        if not isinstance(data_source, DataSource):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "a Python data source instance of type 'DataSource'",
+                    "actual": f"'{type(data_source).__name__}'",
+                },
+            )
+
+        # Receive the data source output schema.
+        schema_json = utf8_deserializer.loads(infile)
+        schema = _parse_datatype_json_string(schema_json)
+        if not isinstance(schema, StructType):
+            raise PySparkAssertionError(
+                error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH",
+                message_parameters={
+                    "expected": "an output schema of type 'StructType'",
+                    "actual": f"'{type(schema).__name__}'",
+                },
+            )
+
+        # Instantiate data source reader.
+        try:
+            reader = data_source.streamReader(schema=schema)
+            # Initialization succeed.
+            write_int(0, outfile)
+            outfile.flush()
+
+            # handle method call from socket
+            while True:
+                func_id = read_int(infile)
+                if func_id == INITIAL_OFFSET_FUNC_ID:
+                    initial_offset_func(reader, outfile)
+                elif func_id == LATEST_OFFSET_FUNC_ID:
+                    latest_offset_func(reader, outfile)
+                elif func_id == PARTITIONS_FUNC_ID:
+                    partitions_func(reader, infile, outfile)
+                elif func_id == COMMIT_FUNC_ID:
+                    commit_func(reader, infile, outfile)
+                outfile.flush()

Review Comment:
   Exception handling added.



##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def initialOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def latestOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod

Review Comment:
   Error logic added.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1520450074


##########
python/pyspark/sql/datasource.py:
##########
@@ -298,6 +320,133 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
         ...
 
 
+class DataSourceStreamReader(ABC):
+    """
+    A base class for streaming data source readers. Data source stream readers are responsible
+    for outputting data from a streaming data source.
+
+    .. versionadded: 4.0.0
+    """
+
+    def initialOffset(self) -> dict:
+        """
+        Return the initial offset of the streaming data source.
+        A new streaming query starts reading data from the initial offset.
+        If Spark is restarting an existing query, it will restart from the check-pointed offset
+        rather than the initial one.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def initialOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "initialOffset"},
+        )
+
+    def latestOffset(self) -> dict:
+        """
+        Returns the most recent offset available.
+
+        Returns
+        -------
+        dict
+            A dict or recursive dict whose key and value are primitive types, which includes
+            Integer, String and Boolean.
+
+        Examples
+        --------
+        >>> def latestOffset(self):
+        ...     return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}}
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "latestOffset"},
+        )
+
+    def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]:
+        """
+        Returns a list of InputPartition  given the start and end offsets. Each InputPartition
+        represents a data split that can be processed by one Spark task.
+
+        Parameters
+        ----------
+        start : dict
+            The start offset of the microbatch to plan partitioning.
+        end : dict
+            The end offset of the microbatch to plan partitioning.
+
+        Returns
+        -------
+        Sequence[InputPartition]
+            A sequence of partitions for this data source. Each partition value
+            must be an instance of `InputPartition` or a subclass of it.
+        """
+        ...
+        raise PySparkNotImplementedError(
+            error_class="NOT_IMPLEMENTED",
+            message_parameters={"feature": "partitions"},
+        )
+
+    @abstractmethod
+    def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:

Review Comment:
   Yes let's make it `Iterator[Tuple]`.



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "allisonwang-db (via GitHub)" <gi...@apache.org>.
allisonwang-db commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1520455477


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.execution.python
+
+import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
+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}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.StructType
+
+object PythonStreamingSourceRunner {
+  // When the python process for python_streaming_source_runner receives one of the
+  // integers below, it will invoke the corresponding function of StreamReader instance.
+  val INITIAL_OFFSET_FUNC_ID = 884
+  val LATEST_OFFSET_FUNC_ID = 885
+  val PARTITIONS_FUNC_ID = 886
+  val COMMIT_FUNC_ID = 887
+}
+
+/**
+ * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM.
+ * A runner spawns a python worker process. In the main function, set up communication
+ * between JVM and python process through socket and create a DataSourceStreamReader instance.
+ * In an infinite loop, the python worker process poll information(function name and parameters)
+ * from the socket, invoke the corresponding method of StreamReader and send return value to JVM.
+ */
+class PythonStreamingSourceRunner(
+    func: PythonFunction,
+    outputSchema: StructType) extends Logging  {
+  val workerModule = "pyspark.sql.streaming.python_streaming_source_runner"
+
+  private val conf = SparkEnv.get.conf
+  private val bufferSize: Int = conf.get(BUFFER_SIZE)
+  private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT)
+
+  private val envVars: java.util.Map[String, String] = func.envVars
+  private val pythonExec: String = func.pythonExec
+  private var pythonWorker: Option[PythonWorker] = None
+  private var pythonWorkerFactory: Option[PythonWorkerFactory] = None
+  private val pythonVer: String = func.pythonVer
+
+  private var dataOut: DataOutputStream = null
+  private var dataIn: DataInputStream = null
+
+  import PythonStreamingSourceRunner._
+
+  /**
+   * Initializes the Python worker for running the streaming source.
+   */
+  def init(): Unit = {
+    logInfo(s"Initializing Python runner 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)
+
+    val prevConf = conf.get(PYTHON_USE_DAEMON)
+    conf.set(PYTHON_USE_DAEMON, false)
+    try {
+      val workerFactory =
+        new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap)
+      val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true)
+      pythonWorker = Some(worker)
+      pythonWorkerFactory = Some(workerFactory)
+    } finally {
+      conf.set(PYTHON_USE_DAEMON, prevConf)
+    }
+
+    val stream = new BufferedOutputStream(
+      pythonWorker.get.channel.socket().getOutputStream, bufferSize)
+    dataOut = new DataOutputStream(stream)
+
+    PythonWorkerUtils.writePythonVersion(pythonVer, dataOut)
+
+    val pythonIncludes = func.pythonIncludes.asScala.toSet
+    PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut)
+
+    // Send the user function to python process
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send output schema
+    PythonWorkerUtils.writeUTF(outputSchema.json, dataOut)
+
+    dataOut.flush()
+
+    dataIn = new DataInputStream(
+      new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize))
+
+    val initStatus = dataIn.readInt()
+    if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryCompilationErrors.pythonDataSourceError(
+        action = "plan", tpe = "initialize source", msg = msg)
+    }
+  }
+
+  /**
+   * Invokes latestOffset() function of the stream reader and receive the return value.
+   */
+  def latestOffset(): String = {
+    dataOut.writeInt(LATEST_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "latestOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes initialOffset() function of the stream reader and receive the return value.
+   */
+  def initialOffset(): String = {
+    dataOut.writeInt(INITIAL_OFFSET_FUNC_ID)
+    dataOut.flush()
+    val len = dataIn.readInt()
+    if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "initialOffset", msg)
+    }
+    PythonWorkerUtils.readUTF(len, dataIn)
+  }
+
+  /**
+   * Invokes partitions(start, end) function of the stream reader and receive the return value.
+   */
+  def partitions(start: String, end: String): Array[Array[Byte]] = {
+    dataOut.writeInt(PARTITIONS_FUNC_ID)
+    PythonWorkerUtils.writeUTF(start, dataOut)
+    PythonWorkerUtils.writeUTF(end, dataOut)
+    dataOut.flush()
+    // Receive the list of partitions, if any.
+    val pickledPartitions = ArrayBuffer.empty[Array[Byte]]
+    val numPartitions = dataIn.readInt()
+    if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError(
+        action = "planPartitions", msg)
+    }
+    for (_ <- 0 until numPartitions) {
+      val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn)
+      pickledPartitions.append(pickledPartition)
+    }
+    pickledPartitions.toArray
+  }
+
+  /**
+   * Invokes commit(end) function of the stream reader and receive the return value.
+   */
+  def commit(end: String): Unit = {
+    dataOut.writeInt(COMMIT_FUNC_ID)

Review Comment:
   The `PythonPlannerRunner` used in batch implementation and each function here will catch errors from the Python process:
   ```
       val length = dataIn.readInt()
       if (length == SpecialLengths.PYTHON_EXCEPTION_THROWN) { // throw exceptions
   ```
   But I am not sure what the behavior is if the Python process dies. cc @ueshin 



-- 
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


Re: [PR] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #45023:
URL: https://github.com/apache/spark/pull/45023#discussion_r1491464124


##########
python/pyspark/sql/streaming/python_streaming_source_runner.py:
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+#
+
+import os
+import sys
+import json
+from typing import IO
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkAssertionError, PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    write_with_length,
+    SpecialLengths,
+)
+from pyspark.sql.datasource import DataSource
+from pyspark.sql.types import (
+    _parse_datatype_json_string,
+    StructType,
+)
+from pyspark.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    read_command,
+    pickleSer,
+    send_accumulator_updates,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+initial_offset_func_id = 884
+latest_offset_func_id = 885
+partitions_func_id = 886
+commit_func_id = 887
+
+
+def initial_offset_func(reader, outfile):
+    offset = reader.initialOffset()

Review Comment:
   We throw unimplemented error.



-- 
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