dtenedor commented on code in PR #43360:
URL: https://github.com/apache/spark/pull/43360#discussion_r1362706226


##########
python/pyspark/sql/datasource.py:
##########
@@ -0,0 +1,184 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+from abc import ABC, abstractmethod
+from typing import Any, Dict, Iterator, Tuple, Union, TYPE_CHECKING
+
+from pyspark.sql import Row
+from pyspark.sql.types import StructType
+
+if TYPE_CHECKING:
+    from pyspark.sql._typing import OptionalPrimitiveType
+
+
+__all__ = ["DataSource", "DataSourceReader"]
+
+
+class DataSource:
+    """
+    A base class for data sources.
+
+    This class represents a custom data source that allows for reading from and
+    writing to it. The data source provides methods to create readers and 
writers
+    for reading and writing data, respectively. At least one of the methods 
`reader`
+    or `writer` must be implemented by any subclass to make the data source 
either
+    readable or writable.
+
+    After implementing this interface, you can start to load your data source 
using
+    `spark.read.format(...).load()` and save data using 
`df.write.format(...).save()`.

Review Comment:
   ```suggestion
       `spark.read.format(...).load()` and save data using 
`df.write.format(...).save()`.
       You can also create tables using SQL with:
       `CREATE TABLE tableName(<columnList>) USING <dataSourceName> OPTIONS 
<options>`
   ```



##########
python/pyspark/sql/datasource.py:
##########
@@ -0,0 +1,184 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+from abc import ABC, abstractmethod
+from typing import Any, Dict, Iterator, Tuple, Union, TYPE_CHECKING
+
+from pyspark.sql import Row
+from pyspark.sql.types import StructType
+
+if TYPE_CHECKING:
+    from pyspark.sql._typing import OptionalPrimitiveType
+
+
+__all__ = ["DataSource", "DataSourceReader"]
+
+
+class DataSource:
+    """
+    A base class for data sources.
+
+    This class represents a custom data source that allows for reading from and

Review Comment:
   ```suggestion
       This class represents a custom data source that allows for reading from 
and/or
   ```



##########
python/pyspark/sql/worker/plan_data_source_read.py:
##########
@@ -0,0 +1,131 @@
+#
+# 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
+from typing import Any, IO, Iterator
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    SpecialLengths,
+    CloudPickleSerializer,
+)
+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_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    """
+    Plan Python data source read.
+    """
+    try:
+        check_python_version(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", 
"-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        setup_spark_files(infile)
+        setup_broadcasts(infile)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+        if not isinstance(data_source, DataSource):
+            raise PySparkRuntimeError(

Review Comment:
   in general we should give error classes to each of these. But it seems this 
should never happen, should we just turn it into an assert instead? Same for 
other `PySparkRuntimeError` cases below.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PythonDataSourceStrategy.scala:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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
+
+import org.apache.spark.api.python.{PythonEvalType, PythonFunction, 
SimplePythonFunction}
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions.PythonUDTF
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkPlan
+import 
org.apache.spark.sql.execution.python.UserDefinedPythonDataSourceReadRunner
+
+object PythonDataSourceStrategy extends Strategy with Logging {
+  def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
+    case p @ logical.PythonDataSource(dataSource: PythonFunction, schema, _) =>
+      val info = new UserDefinedPythonDataSourceReadRunner(dataSource, 
schema).runInPython()
+
+      val readerFunc = SimplePythonFunction(
+        command = info.func.toSeq,
+        envVars = dataSource.envVars,
+        pythonIncludes = dataSource.pythonIncludes,
+        pythonExec = dataSource.pythonExec,
+        pythonVer = dataSource.pythonVer,
+        broadcastVars = dataSource.broadcastVars,
+        accumulator = dataSource.accumulator
+      )
+
+      val partitionPlan = logical.PythonDataSourcePartition(info.partitions)
+
+      // Construct a Python UDTF for the reader function.
+      val pythonUDTF = PythonUDTF(
+        name = "PythonDataSourceReaderUDTF",
+        func = readerFunc,
+        elementSchema = schema,
+        children = partitionPlan.output,
+        evalType = PythonEvalType.SQL_TABLE_UDF,
+        udfDeterministic = false,

Review Comment:
   we should probably add an option for the user to toggle this. This default 
of `false` sounds good.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonDataSourcePartitionExec.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.{InputRDDCodegen, LeafExecNode, 
SQLExecution}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+
+case class PythonDataSourcePartitionExec(

Review Comment:
   please add a class comment?



##########
python/pyspark/sql/readwriter.py:
##########
@@ -303,6 +330,40 @@ def load(
         if schema is not None:
             self.schema(schema)
         self.options(**options)
+
+        # Load a Python data source
+        if isinstance(self._format, Callable):  # type: ignore[arg-type]
+            # TODO(SPARK-45560): support load() with non-empty path.
+
+            # Create an instance of the data source.
+            data_source_cls = cast(Type[DataSource], self._format)
+            data_source = data_source_cls(self._options)
+
+            # Get schema of the data source
+            schema = self._schema or data_source.schema()

Review Comment:
   It will be necessary to inspect the values of the provided options for each 
call in order to determine which schema to generate. This is analogous to the 
Python UDTF `analyze` call we designed which runs on the driver.



##########
python/pyspark/sql/worker/plan_data_source_read.py:
##########
@@ -0,0 +1,131 @@
+#
+# 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
+from typing import Any, IO, Iterator
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    SpecialLengths,
+    CloudPickleSerializer,
+)
+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_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    """
+    Plan Python data source read.

Review Comment:
   can we expand this a bit more to mention when this gets called during the 
query processing sequence, and what it is responsible for receiving as input 
and returning as output?



##########
python/pyspark/sql/worker/plan_data_source_read.py:
##########
@@ -0,0 +1,131 @@
+#
+# 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
+from typing import Any, IO, Iterator
+
+from pyspark.accumulators import _accumulatorRegistry
+from pyspark.errors import PySparkRuntimeError
+from pyspark.java_gateway import local_connect_and_auth
+from pyspark.serializers import (
+    read_int,
+    write_int,
+    SpecialLengths,
+    CloudPickleSerializer,
+)
+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_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+    utf8_deserializer,
+)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    """
+    Plan Python data source read.
+    """
+    try:
+        check_python_version(infile)
+
+        memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", 
"-1"))
+        setup_memory_limits(memory_limit_mb)
+
+        setup_spark_files(infile)
+        setup_broadcasts(infile)
+
+        _accumulatorRegistry.clear()
+
+        # Receive the data source instance.
+        data_source = read_command(pickleSer, infile)
+        if not isinstance(data_source, DataSource):
+            raise PySparkRuntimeError(
+                f"Expected a Python data source instance of type 'DataSource', 
but "
+                f"got '{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 PySparkRuntimeError(
+                f"Expected a Python data source schema of type 'StructType', 
but "
+                f"got '{type(schema).__name__}'."
+            )
+
+        # Instantiate data source reader.
+        try:
+            reader = data_source.reader(schema=schema)
+        except NotImplementedError:
+            raise PySparkRuntimeError(

Review Comment:
   this one is possible to exercise if the data source is implemented 
incorrectly; can we add an error class and a test case? Same for L87 below?



##########
python/pyspark/sql/readwriter.py:
##########
@@ -303,6 +330,40 @@ def load(
         if schema is not None:
             self.schema(schema)
         self.options(**options)
+
+        # Load a Python data source
+        if isinstance(self._format, Callable):  # type: ignore[arg-type]
+            # TODO(SPARK-45560): support load() with non-empty path.
+
+            # Create an instance of the data source.
+            data_source_cls = cast(Type[DataSource], self._format)
+            data_source = data_source_cls(self._options)
+
+            # Get schema of the data source
+            schema = self._schema or data_source.schema()
+            if isinstance(schema, str):
+                dt = _parse_datatype_string(schema)

Review Comment:
   we will want to check for parsing failures here and return a reasonable 
error message.



##########
python/pyspark/sql/datasource.py:
##########
@@ -0,0 +1,184 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+from abc import ABC, abstractmethod
+from typing import Any, Dict, Iterator, Tuple, Union, TYPE_CHECKING
+
+from pyspark.sql import Row
+from pyspark.sql.types import StructType
+
+if TYPE_CHECKING:
+    from pyspark.sql._typing import OptionalPrimitiveType
+
+
+__all__ = ["DataSource", "DataSourceReader"]
+
+
+class DataSource:
+    """
+    A base class for data sources.
+
+    This class represents a custom data source that allows for reading from and
+    writing to it. The data source provides methods to create readers and 
writers
+    for reading and writing data, respectively. At least one of the methods 
`reader`
+    or `writer` must be implemented by any subclass to make the data source 
either
+    readable or writable.

Review Comment:
   ```suggestion
       readable or writable (or both).
   ```



##########
python/pyspark/sql/datasource.py:
##########
@@ -0,0 +1,184 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+from abc import ABC, abstractmethod
+from typing import Any, Dict, Iterator, Tuple, Union, TYPE_CHECKING
+
+from pyspark.sql import Row
+from pyspark.sql.types import StructType
+
+if TYPE_CHECKING:
+    from pyspark.sql._typing import OptionalPrimitiveType
+
+
+__all__ = ["DataSource", "DataSourceReader"]
+
+
+class DataSource:
+    """
+    A base class for data sources.
+
+    This class represents a custom data source that allows for reading from and
+    writing to it. The data source provides methods to create readers and 
writers
+    for reading and writing data, respectively. At least one of the methods 
`reader`
+    or `writer` must be implemented by any subclass to make the data source 
either
+    readable or writable.
+
+    After implementing this interface, you can start to load your data source 
using
+    `spark.read.format(...).load()` and save data using 
`df.write.format(...).save()`.
+    """
+
+    def __init__(self, options: Dict[str, "OptionalPrimitiveType"]):
+        """
+        Initializes the data source with user-provided options.
+
+        Parameters
+        ----------
+        options : dict
+            A dictionary representing the options for this data source.
+
+        Notes
+        -----
+        This method should not contain any non-serializable objects.
+        """
+        self.options = options
+
+    @property
+    def name(self) -> str:
+        """
+        Returns a string represents the short name of this data source.
+        """
+        return self.__class__.__name__
+
+    def schema(self) -> Union[StructType, str]:
+        """
+        Returns the schema of the data source.
+
+        It can reference the `options` field to infer the data source's schema 
when
+        users do not explicitly specify it. This method is invoked once when 
calling
+        `spark.read.format(...).load()` to get the schema for a data source 
read operation.
+        If this method is not implemented, and a user does not provide a 
schema when
+        reading the data source, an exception will be thrown.
+
+        Returns
+        -------
+        schema : StructType or str
+            The schema of this data source or a DDL string represents the 
schema
+
+        Examples
+        --------
+        Returns a DDL string:
+
+        >>> def schema(self):
+        >>>    return "a INT, b STRING"

Review Comment:
   can we add test cases where we send the schema as a string instead of 
StructType, and a positive case as well as a negative case where it doesn't 
parse successfully with fromDDL?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonDataSource.scala:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.python
+
+import java.io.{DataInputStream, DataOutputStream}
+
+import scala.collection.mutable.ArrayBuffer
+
+import net.razorvine.pickle.Pickler
+
+import org.apache.spark.api.python.{PythonFunction, PythonWorkerUtils, 
SpecialLengths}
+import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.PythonDataSource
+import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.StructType
+
+/**
+ * A user-defined Python data source. This is used by the Python API.
+ */
+case class UserDefinedPythonDataSource(
+    dataSource: PythonFunction,
+    schema: StructType) {
+  def apply(session: SparkSession): DataFrame = {
+    val source = PythonDataSource(dataSource, schema, output = 
toAttributes(schema))
+    Dataset.ofRows(session, source)
+  }
+}
+
+case class PythonDataSourceReadInfo(
+    func: Array[Byte],
+    partitions: Seq[Array[Byte]])
+
+class UserDefinedPythonDataSourceReadRunner(
+    func: PythonFunction,
+    schema: StructType) extends 
PythonPlannerRunner[PythonDataSourceReadInfo](func) {
+
+  override val workerModule = "pyspark.sql.worker.plan_data_source_read"
+
+  override protected def writeToPython(dataOut: DataOutputStream, pickler: 
Pickler): Unit = {
+    // Send Python data source
+    PythonWorkerUtils.writePythonFunction(func, dataOut)
+
+    // Send schema
+    PythonWorkerUtils.writeUTF(schema.json, dataOut)
+  }
+
+  override protected def receiveFromPython(dataIn: DataInputStream): 
PythonDataSourceReadInfo = {
+    // Receive the picked reader or an exception raised in Python worker.
+    val length = dataIn.readInt()
+    if (length == SpecialLengths.PYTHON_EXCEPTION_THROWN) {
+      val msg = PythonWorkerUtils.readUTF(dataIn)
+      throw 
QueryCompilationErrors.tableValuedFunctionFailedToAnalyseInPythonError(msg)

Review Comment:
   can we test it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to