ueshin commented on code in PR #43360: URL: https://github.com/apache/spark/pull/43360#discussion_r1358824516
########## sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonDataSource.scala: ########## @@ -0,0 +1,90 @@ +/* + * 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) + } + + // Receive the pickled 'read' function. + val pickledFunction: Array[Byte] = { + val obj = new Array[Byte](length) + dataIn.readFully(obj) + obj + } Review Comment: ```suggestion val pickledFunction = PythonWorkerUtils.readBytes(length, dataIn) ``` ########## python/pyspark/sql/readwriter.py: ########## @@ -303,6 +329,50 @@ def load( if schema is not None: self.schema(schema) self.options(**options) + + # Load a Python data source + if isinstance(self._format, Callable): + # TODO: support path in options. + + # 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): + schema = _parse_datatype_string(schema) + # Check if the schema is a valid StructType. + if not isinstance(schema, StructType): + raise PySparkTypeError( + error_class="NOT_STR_OR_STRUCT", + message_parameters={ + "arg_name": "schema", + "arg_type": type(schema).__name__, + }, + ) + + jschema = self._spark._jsparkSession.parseDataType(schema.json()) + sc = self._spark._sc + pickled_command, broadcast_vars, env, includes = _prepare_for_python_RDD( + sc, data_source + ) + assert sc._jvm is not None + func = sc._jvm.SimplePythonFunction( + bytearray(pickled_command), + env, + includes, + sc.pythonExec, + sc.pythonVer, + broadcast_vars, + sc._javaAccumulator, + ) Review Comment: Can't we use `_wrap_function` in `udf.py`? ########## python/pyspark/sql/readwriter.py: ########## @@ -303,6 +329,50 @@ def load( if schema is not None: self.schema(schema) self.options(**options) + + # Load a Python data source + if isinstance(self._format, Callable): Review Comment: Can't this be `DataSource` instead of `Callable`? ########## 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( + output: Seq[Attribute], + partitions: Seq[Array[Byte]]) + extends LeafExecNode with InputRDDCodegen with PythonSQLMetrics { Review Comment: nit: style; 2 white spaces for `extends`. ########## python/pyspark/sql/tests/test_python_datasource.py: ########## @@ -0,0 +1,85 @@ +# +# 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 unittest + +from pyspark.sql.types import ( + IntegerType, + Row, + StructField, + StructType, +) +from pyspark.testing import assertDataFrameEqual +from pyspark.sql.datasource import DataSource, DataSourceReader +from pyspark.testing.sqlutils import ReusedSQLTestCase + + +class TestDataSourceReader(DataSourceReader): + def read(self, partition): + yield (partition, 0) + yield (partition, 1) + + +class DataSourcePartitionReader(TestDataSourceReader): + def partitions(self): + return [1, 2] + + +class TestDataSource(DataSource): + def schema(self): + return StructType([StructField("id", IntegerType()), StructField("value", IntegerType())]) + + def reader(self, schema): + return TestDataSourceReader() + + +class TestPartitionedDataSource(TestDataSource): + def reader(self, schema): + return DataSourcePartitionReader() + + +class BasePythonDataSourceTestsMixin: + def test_data_source_read(self): + df = self.spark.read.format(TestDataSource).load() + assertDataFrameEqual(df, [Row(id=None, value=0), Row(id=None, value=1)]) + + # TODO: support read with schema Review Comment: Shall we have a JIRA ID for TODO? ```py # TODO(SPARK-xxx): ... ``` -- 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]
