You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "HyukjinKwon (via GitHub)" <gi...@apache.org> on 2023/12/27 04:47:44 UTC

[PR] [SPARK-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

HyukjinKwon opened a new pull request, #44504:
URL: https://github.com/apache/spark/pull/44504

   ### What changes were proposed in this pull request?
   
   This PR proposes to add the support of automatic Python Data Source registration.
   
   **End user perspective:**
   
   ```bash
   # Assume that `customsource` defined a short name as `custom` 
   pip install pyspark_customsource
   ```
   
   Users can directly use the Python Data Source
   
   ```python
   df = spark.format("custom").load()
   ```
   
   **Developer perspective:**
   
   The packages should follow the structure below:
   - The package name should start with `pyspark_` prefix
   - `pyspark_*.DefaultSource` has to be defined that inherits `pyspark.sql.datasource.DataSource`
   
   For example:
   
   ```
   pyspark_customsource
   ├── __init__.py
    ...
   ```
   
   `__init__.py`:
   
   ```python
   from pyspark.sql.datasource import DataSource
   
   class DefaultSource(Datasource):
       pass
   ```
   
   ### Why are the changes needed?
   
   This allows the developers to release and maintain their 3rd party Python Data Sources separately (e.g., in PyPI), and end users can easily install the Python Data Source without doing anything other than just `pip install pyspark_their_source`
   
   ### Does this PR introduce _any_ user-facing change?
   
   Yes, this allows users to `pip install pyspark_custom_source`, and automatically register it as Data Source available in Spark. 
   
   ### How was this patch tested?
   
   TBD
   
   ### 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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

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


##########
core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala:
##########
@@ -145,4 +149,48 @@ private[spark] object PythonUtils extends Logging {
       listOfPackages.foreach(x => logInfo(s"List of Python packages :- ${formatOutput(x)}"))
     }
   }
+
+  // Only for testing.
+  private[spark] var additionalTestingPath: Option[String] = None
+
+  private[spark] def createPythonFunction(command: Array[Byte]): SimplePythonFunction = {
+    val pythonExec: String = sys.env.getOrElse(
+      "PYSPARK_DRIVER_PYTHON", sys.env.getOrElse("PYSPARK_PYTHON", "python3"))
+
+    val sourcePython = if (Utils.isTesting) {
+      // Put PySpark source code instead of the build zip archive so we don't need
+      // to build PySpark every time during development.
+      val sparkHome: String = {
+        require(
+          sys.props.contains("spark.test.home") || sys.env.contains("SPARK_HOME"),
+          "spark.test.home or SPARK_HOME is not set.")
+        sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME"))
+      }
+      val sourcePath = Paths.get(sparkHome, "python").toAbsolutePath
+      val py4jPath = Paths.get(
+        sparkHome, "python", "lib", PythonUtils.PY4J_ZIP_NAME).toAbsolutePath

Review Comment:
   We don't use it but we need it when we import .. e.g., https://github.com/apache/spark/blob/master/python/pyspark/__init__.py#L53 -> https://github.com/apache/spark/blob/master/python/pyspark/conf.py#L23



-- 
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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

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

   Oops, there's a bit of more fixes to make (although the basic cases work). Let me mark as a draft for now.


-- 
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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon closed pull request #44504: [SPARK-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup
URL: https://github.com/apache/spark/pull/44504


-- 
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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

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


##########
python/pyspark/sql/worker/lookup_data_sources.py:
##########
@@ -0,0 +1,99 @@
+#
+# 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 importlib import import_module
+from pkgutil import iter_modules
+import os
+import sys
+from typing import IO
+
+from pyspark.accumulators import _accumulatorRegistry
+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.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    pickleSer,
+    send_accumulator_updates,
+    setup_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    """
+    Main method for looking up the available Python Data Sources in Python path.
+
+    This process is invoked from the `UserDefinedPythonDataSourceLookupRunner.runInPython`
+    method in `UserDefinedPythonDataSource.lookupAllDataSourcesInPython` when the first
+    call related to Python Data Source happens via `DataSourceManager`.
+
+    This is responsible for searching the available Python Data Sources so they can be
+    statically registered automatically.
+    """
+    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()
+
+        infos = {}
+        for info in iter_modules():
+            if info.name.startswith("pyspark_"):
+                mod = import_module(info.name)
+                if hasattr(mod, "DefaultSource") and isinstance(mod.DefaultSource, DataSource):

Review Comment:
   argh



-- 
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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

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

   Thx, let me fix up together at https://github.com/apache/spark/pull/44519


-- 
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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

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

   @HyukjinKwon 
   I have reverted https://github.com/apache/spark/pull/44504 (CommitID: `229a4eaf547e5c263c749bd53f7f9a89f4a9bea9`). 
   Based on the current running results, the `Run Spark on Kubernetes Integration test` failure of GA is related to 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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

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


##########
core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala:
##########
@@ -145,4 +149,48 @@ private[spark] object PythonUtils extends Logging {
       listOfPackages.foreach(x => logInfo(s"List of Python packages :- ${formatOutput(x)}"))
     }
   }
+
+  // Only for testing.
+  private[spark] var additionalTestingPath: Option[String] = None
+
+  private[spark] def createPythonFunction(command: Array[Byte]): SimplePythonFunction = {
+    val pythonExec: String = sys.env.getOrElse(
+      "PYSPARK_DRIVER_PYTHON", sys.env.getOrElse("PYSPARK_PYTHON", "python3"))
+
+    val sourcePython = if (Utils.isTesting) {
+      // Put PySpark source code instead of the build zip archive so we don't need
+      // to build PySpark every time during development.
+      val sparkHome: String = {
+        require(
+          sys.props.contains("spark.test.home") || sys.env.contains("SPARK_HOME"),
+          "spark.test.home or SPARK_HOME is not set.")
+        sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME"))
+      }
+      val sourcePath = Paths.get(sparkHome, "python").toAbsolutePath
+      val py4jPath = Paths.get(
+        sparkHome, "python", "lib", PythonUtils.PY4J_ZIP_NAME).toAbsolutePath

Review Comment:
   We don't use it but we need it when we import .. e.g., https://github.com/apache/spark/blob/master/python/pyspark/__init__.py#L53 -> https://github.com/apache/spark/blob/master/python/pyspark/conf.py#L23 when we import Python Data Source.



-- 
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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

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

   Let me actually add the test cases here together while I am here.


-- 
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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

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


##########
python/pyspark/sql/worker/lookup_data_sources.py:
##########
@@ -0,0 +1,99 @@
+#
+# 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 importlib import import_module
+from pkgutil import iter_modules
+import os
+import sys
+from typing import IO
+
+from pyspark.accumulators import _accumulatorRegistry
+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.util import handle_worker_exception
+from pyspark.worker_util import (
+    check_python_version,
+    pickleSer,
+    send_accumulator_updates,
+    setup_broadcasts,
+    setup_memory_limits,
+    setup_spark_files,
+)
+
+
+def main(infile: IO, outfile: IO) -> None:
+    """
+    Main method for looking up the available Python Data Sources in Python path.
+
+    This process is invoked from the `UserDefinedPythonDataSourceLookupRunner.runInPython`
+    method in `UserDefinedPythonDataSource.lookupAllDataSourcesInPython` when the first
+    call related to Python Data Source happens via `DataSourceManager`.
+
+    This is responsible for searching the available Python Data Sources so they can be
+    statically registered automatically.
+    """
+    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()
+
+        infos = {}
+        for info in iter_modules():
+            if info.name.startswith("pyspark_"):
+                mod = import_module(info.name)
+                if hasattr(mod, "DefaultSource") and isinstance(mod.DefaultSource, DataSource):

Review Comment:
   `issubclass` instead of `isinstance`?



-- 
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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

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

   Merged 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-45917][PYTHON][SQL] Automatic registration of Python Data Source on startup [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonDataSource.scala:
##########
@@ -404,15 +404,68 @@ object UserDefinedPythonDataSource {
    * The schema of the output to the Python data source write function.
    */
   val writeOutputSchema: StructType = new StructType().add("message", BinaryType)
+
+  /**
+   * (Driver-side) Look up all available Python Data Sources.
+   */
+  def lookupAllDataSourcesInPython(): PythonAllDataSourcesCreationResult = {
+    new UserDefinedPythonDataSourceLookupRunner(
+      PythonUtils.createPythonFunction(Array.empty[Byte])).runInPython()
+  }
 }
 
+/**
+ * All Data Sources in Python
+ */
+case class PythonAllDataSourcesCreationResult(
+    names: Array[String], dataSources: Array[Array[Byte]])
+
 /**
  * Used to store the result of creating a Python data source in the Python process.
  */
 case class PythonDataSourceCreationResult(
     dataSource: Array[Byte],
     schema: StructType)
 
+/**
+ * A runner used to look up Python Data Sources available in Python path.
+ */
+class UserDefinedPythonDataSourceLookupRunner(lookupSources: PythonFunction)
+    extends PythonPlannerRunner[PythonAllDataSourcesCreationResult](lookupSources) {
+
+  override val workerModule = "pyspark.sql.worker.lookup_data_sources"
+
+  override protected def writeToPython(dataOut: DataOutputStream, pickler: Pickler): Unit = {
+    PythonWorkerUtils.writePythonFunction(lookupSources, dataOut)

Review Comment:
   We don't need to send anything here? Seems like there is not a corresponding read in the Python worker.



##########
core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala:
##########
@@ -145,4 +149,48 @@ private[spark] object PythonUtils extends Logging {
       listOfPackages.foreach(x => logInfo(s"List of Python packages :- ${formatOutput(x)}"))
     }
   }
+
+  // Only for testing.
+  private[spark] var additionalTestingPath: Option[String] = None
+
+  private[spark] def createPythonFunction(command: Array[Byte]): SimplePythonFunction = {
+    val pythonExec: String = sys.env.getOrElse(
+      "PYSPARK_DRIVER_PYTHON", sys.env.getOrElse("PYSPARK_PYTHON", "python3"))
+
+    val sourcePython = if (Utils.isTesting) {
+      // Put PySpark source code instead of the build zip archive so we don't need
+      // to build PySpark every time during development.
+      val sparkHome: String = {
+        require(
+          sys.props.contains("spark.test.home") || sys.env.contains("SPARK_HOME"),
+          "spark.test.home or SPARK_HOME is not set.")
+        sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME"))
+      }
+      val sourcePath = Paths.get(sparkHome, "python").toAbsolutePath
+      val py4jPath = Paths.get(
+        sparkHome, "python", "lib", PythonUtils.PY4J_ZIP_NAME).toAbsolutePath

Review Comment:
   Do we need Py4J path? The Python functions are not supposed to use Py4J?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala:
##########
@@ -40,6 +44,52 @@ class PythonDataSourceSuite extends QueryTest with SharedSparkSession {
       |        yield (1, partition.value)
       |        yield (2, partition.value)
       |""".stripMargin
+  private val staticSourceName = "custom_source"
+  private var tempDir: File = _
+
+  override def beforeAll(): Unit = {
+    // Create a Python Data Source package before starting up the Spark Session
+    // that triggers automatic registration of the Python Data Source.
+    val dataSourceScript =
+    s"""
+       |from pyspark.sql.datasource import DataSource, DataSourceReader
+       |$simpleDataSourceReaderScript
+       |
+       |class DefaultSource(DataSource):
+       |    def schema(self) -> str:
+       |        return "id INT, partition INT"
+       |
+       |    def reader(self, schema):
+       |        return SimpleDataSourceReader()
+       |
+       |    @classmethod
+       |    def name(cls):
+       |        return "$staticSourceName"
+       |""".stripMargin
+    tempDir = Utils.createTempDir()
+    // Write a temporary package to test.
+    // tmp/my_source
+    // tmp/my_source/__init__.py
+    val packageDir = new File(tempDir, "pyspark_mysource")
+    assert(packageDir.mkdir())
+    Utils.tryWithResource(
+      new FileWriter(new File(packageDir, "__init__.py")))(_.write(dataSourceScript))
+    // So Spark Session initialization can lookup this temporary directory.
+    PythonUtils.additionalTestingPath = Some(tempDir.toString)
+    super.beforeAll()
+  }
+
+  override def afterAll(): Unit = {
+    Utils.deleteRecursively(tempDir)
+    PythonUtils.additionalTestingPath = None
+    super.afterAll()

Review Comment:
   just in case,
   
   ```suggestion
       try {
         Utils.deleteRecursively(tempDir)
         PythonUtils.additionalTestingPath = None
       } finally {
         super.afterAll()
       }
   ```
   



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