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-unsubscr...@spark.apache.org

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


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

Reply via email to