-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-45597][PYTHON][SQL] Support creating table using a Python data source in SQL (DSv2 exec) #44305
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 (DSv2 exec) #44305
Changes from all commits
8be3206
82e8ec7
5497b9f
6e1a9f1
0251314
a9d1e1c
8846bf5
7a23cb4
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 |
|---|---|---|
|
|
@@ -780,7 +780,7 @@ class SparkSession private( | |
| DataSource.lookupDataSource(runner, sessionState.conf) match { | ||
| case source if classOf[ExternalCommandRunner].isAssignableFrom(source) => | ||
| Dataset.ofRows(self, ExternalCommandExecutor( | ||
| source.getDeclaredConstructor().newInstance() | ||
| DataSource.newDataSourceInstance(runner, source) | ||
|
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. It may be arguable that if this is a breaking change. Now people need to worry about python data source in the code that is to deal with DS v1 only.
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.
|
||
| .asInstanceOf[ExternalCommandRunner], command, options)) | ||
|
|
||
| case _ => | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -44,6 +44,7 @@ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat | |
| import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 | ||
| import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 | ||
| import org.apache.spark.sql.execution.datasources.xml.XmlFileFormat | ||
| import org.apache.spark.sql.execution.python.PythonTableProvider | ||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} | ||
| import org.apache.spark.sql.internal.SQLConf | ||
|
|
@@ -105,13 +106,14 @@ case class DataSource( | |
| // [[FileDataSourceV2]] will still be used if we call the load()/save() method in | ||
| // [[DataFrameReader]]/[[DataFrameWriter]], since they use method `lookupDataSource` | ||
| // instead of `providingClass`. | ||
| cls.getDeclaredConstructor().newInstance() match { | ||
| DataSource.newDataSourceInstance(className, cls) match { | ||
| case f: FileDataSourceV2 => f.fallbackFileFormat | ||
|
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. and here too
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. and if (DDLUtils.isDatasourceTable(catalogTable)) {
DataSource.newDataSourceInstance(
catalogTable.provider.get,
DataSource.lookupDataSource(catalogTable.provider.get, conf)) match {
// For datasource table, this command can only support the following File format.
// TextFileFormat only default to one column "value"
// Hive type is already considered as hive serde table, so the logic will not
// come in here.
case _: CSVFileFormat | _: JsonFileFormat | _: ParquetFileFormat =>
case _: JsonDataSourceV2 | _: CSVDataSourceV2 |
_: OrcDataSourceV2 | _: ParquetDataSourceV2 =>
case s if s.getClass.getCanonicalName.endsWith("OrcFileFormat") =>
case s =>
throw QueryCompilationErrors.alterAddColNotSupportDatasourceTableError(s, table)
}
}
catalogTable
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. and DataStreamReader: val v1DataSource = DataSource(
sparkSession,
userSpecifiedSchema = userSpecifiedSchema,
className = source,
options = optionsWithPath.originalMap)
val v1Relation = ds match {
case _: StreamSourceProvider => Some(StreamingRelation(v1DataSource))
case _ => None
}
ds match {
// file source v2 does not support streaming yet.
case provider: TableProvider if !provider.isInstanceOf[FileDataSourceV2] =>
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. and DataStreamWriter: val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf)
val disabledSources =
Utils.stringToSeq(df.sparkSession.sessionState.conf.disabledV2StreamingWriters)
val useV1Source = disabledSources.contains(cls.getCanonicalName) ||
// file source v2 does not support streaming yet.
classOf[FileDataSourceV2].isAssignableFrom(cls)
val optionsWithPath = if (path.isEmpty) {
extraOptions
} else {
extraOptions + ("path" -> path.get)
}
val sink = if (classOf[TableProvider].isAssignableFrom(cls) && !useV1Source) { |
||
| case _ => cls | ||
| } | ||
| } | ||
|
|
||
| private[sql] def providingInstance(): Any = providingClass.getConstructor().newInstance() | ||
| private[sql] def providingInstance(): Any = | ||
| DataSource.newDataSourceInstance(className, providingClass) | ||
|
|
||
| private def newHadoopConfiguration(): Configuration = | ||
| sparkSession.sessionState.newHadoopConfWithOptions(options) | ||
|
|
@@ -622,6 +624,15 @@ object DataSource extends Logging { | |
| "org.apache.spark.sql.sources.HadoopFsRelationProvider", | ||
| "org.apache.spark.Logging") | ||
|
|
||
| /** Create the instance of the datasource */ | ||
| def newDataSourceInstance(provider: String, providingClass: Class[_]): Any = { | ||
| providingClass match { | ||
| case cls if classOf[PythonTableProvider].isAssignableFrom(cls) => | ||
| cls.getDeclaredConstructor(classOf[String]).newInstance(provider) | ||
| case cls => cls.getDeclaredConstructor().newInstance() | ||
| } | ||
| } | ||
|
|
||
| /** Given a provider name, look up the data source class definition. */ | ||
| def lookupDataSource(provider: String, conf: SQLConf): Class[_] = { | ||
| val provider1 = backwardCompatibilityMap.getOrElse(provider, provider) match { | ||
|
|
@@ -649,6 +660,9 @@ object DataSource extends Logging { | |
| // Found the data source using fully qualified path | ||
| dataSource | ||
| case Failure(error) => | ||
| // TODO(SPARK-45600): should be session-based. | ||
| val isUserDefinedDataSource = SparkSession.getActiveSession.exists( | ||
| _.sessionState.dataSourceManager.dataSourceExists(provider)) | ||
| if (provider1.startsWith("org.apache.spark.sql.hive.orc")) { | ||
| throw QueryCompilationErrors.orcNotUsedWithHiveEnabledError() | ||
| } else if (provider1.toLowerCase(Locale.ROOT) == "avro" || | ||
|
|
@@ -657,6 +671,8 @@ object DataSource extends Logging { | |
| throw QueryCompilationErrors.failedToFindAvroDataSourceError(provider1) | ||
| } else if (provider1.toLowerCase(Locale.ROOT) == "kafka") { | ||
| throw QueryCompilationErrors.failedToFindKafkaDataSourceError(provider1) | ||
| } else if (isUserDefinedDataSource) { | ||
| classOf[PythonTableProvider] | ||
| } else { | ||
| throw QueryExecutionErrors.dataSourceNotFoundError(provider1, error) | ||
| } | ||
|
|
@@ -673,6 +689,14 @@ object DataSource extends Logging { | |
| } | ||
| case head :: Nil => | ||
| // there is exactly one registered alias | ||
| // TODO(SPARK-45600): should be session-based. | ||
| val isUserDefinedDataSource = SparkSession.getActiveSession.exists( | ||
| _.sessionState.dataSourceManager.dataSourceExists(provider)) | ||
| // The source can be successfully loaded as either a V1 or a V2 data source. | ||
| // Check if it is also a user-defined data source. | ||
| if (isUserDefinedDataSource) { | ||
| throw QueryCompilationErrors.foundMultipleDataSources(provider) | ||
| } | ||
| head.getClass | ||
| case sources => | ||
| // There are multiple registered aliases for the input. If there is single datasource | ||
|
|
@@ -708,17 +732,18 @@ object DataSource extends Logging { | |
| def lookupDataSourceV2(provider: String, conf: SQLConf): Option[TableProvider] = { | ||
|
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. how do you think of my idea to only put python data source handing in this method? https://github.com/apache/spark/pull/44269/files#diff-2a3ed194aac77f3de25418a74a756d8d821feb2b3d38f4fec144f312e022801aR709
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. yeah I like that idea. Can I do it in a followup though? I would like to extract some changes from your PR, and make another PR.
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. It's not a followup... I have a concern about changing
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. oh okie dokie. I was actually thinking about porting more changes in your PR. I will fix that one alone here for now. |
||
| val useV1Sources = conf.getConf(SQLConf.USE_V1_SOURCE_LIST).toLowerCase(Locale.ROOT) | ||
| .split(",").map(_.trim) | ||
| val cls = lookupDataSource(provider, conf) | ||
| val providingClass = lookupDataSource(provider, conf) | ||
| val instance = try { | ||
| cls.getDeclaredConstructor().newInstance() | ||
| newDataSourceInstance(provider, providingClass) | ||
| } catch { | ||
| // Throw the original error from the data source implementation. | ||
| case e: java.lang.reflect.InvocationTargetException => throw e.getCause | ||
| } | ||
| instance match { | ||
| case d: DataSourceRegister if useV1Sources.contains(d.shortName()) => None | ||
| case t: TableProvider | ||
| if !useV1Sources.contains(cls.getCanonicalName.toLowerCase(Locale.ROOT)) => | ||
| if !useV1Sources.contains( | ||
| providingClass.getCanonicalName.toLowerCase(Locale.ROOT)) => | ||
| Some(t) | ||
| case _ => None | ||
| } | ||
|
|
||
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.
Actually @cloud-fan that would not work .. E.g., if PythonDataSource implements
ExternalCommandRunner, we should load it here.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.
lemme fix it separately. Reading the code path, I think it won't more and less affect.
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.
Let's worry about it when we actually adding this ability to the python data source. We may never add it for simplicity.