-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-45597][PYTHON][SQL] Support creating table using a Python data source in SQL (single wrapper) #44233
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-45597][PYTHON][SQL] Support creating table using a Python data source in SQL (single wrapper) #44233
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,12 +20,22 @@ package org.apache.spark.sql.execution.datasources | |
| import java.util.Locale | ||
| import java.util.concurrent.ConcurrentHashMap | ||
|
|
||
| import scala.jdk.CollectionConverters._ | ||
|
|
||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession, SQLContext} | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| 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.{Scan, ScanBuilder, V1Scan} | ||
| import org.apache.spark.sql.errors.QueryCompilationErrors | ||
| import org.apache.spark.sql.sources.{BaseRelation, TableScan} | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
|
||
|
|
||
| /** | ||
| * A manager for user-defined data sources. It is used to register and lookup data sources by | ||
|
|
@@ -40,6 +50,8 @@ class DataSourceManager extends Logging { | |
| CaseInsensitiveMap[String] // options | ||
| ) => LogicalPlan | ||
|
|
||
| // TODO(SPARK-45917): Statically load Python Data Source so idempotently Python | ||
| // Data Sources can be loaded even when the Driver is restarted. | ||
| private val dataSourceBuilders = new ConcurrentHashMap[String, DataSourceBuilder]() | ||
|
|
||
| private def normalize(name: String): String = name.toLowerCase(Locale.ROOT) | ||
|
|
@@ -81,3 +93,60 @@ class DataSourceManager extends Logging { | |
| manager | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Data Source V2 wrapper for Python Data Source. | ||
| */ | ||
| class PythonTableProvider(shortName: String) extends TableProvider { | ||
| private var sourceDataFrame: DataFrame = _ | ||
|
|
||
| private def getOrCreateSourceDataFrame( | ||
| options: CaseInsensitiveStringMap, maybeSchema: Option[StructType]): DataFrame = { | ||
| if (sourceDataFrame != null) return sourceDataFrame | ||
| // TODO(SPARK-45600): should be session-based. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This one should be fixed?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For basic support, I think so. The thing is that we should take a look into session inheritance, testcase, etc. So I leave this as a todo for now. |
||
| val builder = SparkSession.active.sessionState.dataSourceManager.lookupDataSource(shortName) | ||
| val plan = builder( | ||
| SparkSession.active, | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Does it get the correct session for spark connect?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes |
||
| shortName, | ||
| maybeSchema, | ||
| CaseInsensitiveMap(options.asCaseSensitiveMap().asScala.toMap)) | ||
| sourceDataFrame = Dataset.ofRows(SparkSession.active, plan) | ||
| sourceDataFrame | ||
| } | ||
|
|
||
| override def inferSchema(options: CaseInsensitiveStringMap): StructType = | ||
| getOrCreateSourceDataFrame(options, None).schema | ||
|
|
||
| override def getTable( | ||
| schema: StructType, | ||
| partitioning: Array[Transform], | ||
| properties: java.util.Map[String, String]): Table = { | ||
| val givenSchema = 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 V1Scan { | ||
| override def build(): Scan = this | ||
| override def toV1TableScan[T <: BaseRelation with TableScan]( | ||
| context: SQLContext): T = { | ||
| new BaseRelation with TableScan { | ||
| // Avoid Row <> InternalRow conversion | ||
| override val needConversion: Boolean = false | ||
| override def buildScan(): RDD[Row] = | ||
| getOrCreateSourceDataFrame(options, Some(givenSchema)) | ||
| .queryExecution.toRdd.asInstanceOf[RDD[Row]] | ||
| override def schema: StructType = givenSchema | ||
| override def sqlContext: SQLContext = context | ||
| }.asInstanceOf[T] | ||
| } | ||
| override def readSchema(): StructType = givenSchema | ||
| } | ||
| } | ||
|
|
||
| override def schema(): StructType = givenSchema | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we support external metadata for this data source? I.e users can create a table using a python datasource with user defined table schema.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I believe it already does (?).