allisonwang-db commented on code in PR #44305:
URL: https://github.com/apache/spark/pull/44305#discussion_r1427738441


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonDataSource.scala:
##########
@@ -20,58 +20,199 @@ package org.apache.spark.sql.execution.python
 import java.io.{DataInputStream, DataOutputStream}
 
 import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
 
 import net.razorvine.pickle.Pickler
 
-import org.apache.spark.api.python.{PythonFunction, PythonWorkerUtils, 
SimplePythonFunction, SpecialLengths}
-import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
-import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
PythonDataSource}
+import org.apache.spark.JobArtifactSet
+import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType, 
PythonFunction, PythonWorkerUtils, SimplePythonFunction, SpecialLengths}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.PythonUDF
 import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
 import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
+import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, 
TableCapability, TableProvider}
+import org.apache.spark.sql.connector.catalog.TableCapability.BATCH_READ
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, 
PartitionReader, PartitionReaderFactory, Scan, ScanBuilder}
 import org.apache.spark.sql.errors.QueryCompilationErrors
 import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.{DataType, StructType}
+import org.apache.spark.sql.types.{BinaryType, DataType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
 import org.apache.spark.util.ArrayImplicits._
 
+/**
+ * Data Source V2 wrapper for Python Data Source.
+ */
+class PythonTableProvider(shortName: String) extends TableProvider {
+  private var dataSourceInPython: PythonDataSourceCreationResult = _
+  private[this] val jobArtifactUUID = 
JobArtifactSet.getCurrentJobArtifactState.map(_.uuid)
+  private lazy val source: UserDefinedPythonDataSource =
+    
SparkSession.active.sessionState.dataSourceManager.lookupDataSource(shortName)
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    if (dataSourceInPython == null) {
+      dataSourceInPython = source.createDataSourceInPython(shortName, options, 
None)
+    }
+    dataSourceInPython.schema
+  }
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: java.util.Map[String, String]): Table = {
+    val outputSchema = schema
+    new Table with SupportsRead {
+      override def name(): String = shortName
+
+      override def capabilities(): java.util.Set[TableCapability] = 
java.util.EnumSet.of(
+        BATCH_READ)
+
+      override def newScanBuilder(options: CaseInsensitiveStringMap): 
ScanBuilder = {
+        new ScanBuilder with Batch with Scan {
+
+          private lazy val infoInPython: PythonDataSourceReadInfo = {
+            if (dataSourceInPython == null) {
+              dataSourceInPython = source
+                .createDataSourceInPython(shortName, options, 
Some(outputSchema))
+            }
+            source.createReadInfoInPython(dataSourceInPython, outputSchema)
+          }
+
+          override def build(): Scan = this
+
+          override def toBatch: Batch = this
+
+          override def readSchema(): StructType = outputSchema
+
+          override def planInputPartitions(): Array[InputPartition] =
+            infoInPython.partitions.zipWithIndex.map(p => 
PythonInputPartition(p._2, p._1)).toArray
+
+          override def createReaderFactory(): PartitionReaderFactory = {
+            val readerFunc = infoInPython.func
+            new PythonPartitionReaderFactory(
+              source, readerFunc, outputSchema, jobArtifactUUID)
+          }
+        }
+      }
+
+      override def schema(): StructType = outputSchema
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = true

Review Comment:
   I think we can set this to false (default) for now to disallow specifying 
schema. It's much easier to implement the data source without considering the 
user-specify schema.



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