HyukjinKwon commented on code in PR #44305:
URL: https://github.com/apache/spark/pull/44305#discussion_r1426971568


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonDataSource.scala:
##########
@@ -20,58 +20,200 @@ 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, 
BATCH_WRITE}
+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 = {
+    assert(partitioning.isEmpty)
+    val outputSchema = schema
+    new Table with SupportsRead {

Review Comment:
   Actually I intentionally put it together because we should cache 
`dataSourceInPython` executed from the Python worker (that contains both schema 
and pickled datasource), once for schema inference, and once for actual 
reading. So it becomes more readable, and localize the scope of the cache. In 
addition, I think we won't likely extend this Python Table class/instance.



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