allisonwang-db commented on code in PR #44305: URL: https://github.com/apache/spark/pull/44305#discussion_r1427730595
########## 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 am actually thinking about whether we should expose this as an API in Python data source. If a data source cannot handle external metadata, then `.schema(....)` or `CREATE TABLE table(...)` should fail, instead of failing when executing the query. But I am not sure if this will make the Python API too complicated. WDTY? -- 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