-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-32896][SS] Add DataStreamWriter.saveAsTable API #29767
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
Changes from 3 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 |
|---|---|---|
|
|
@@ -27,7 +27,7 @@ import org.apache.spark.api.java.function.VoidFunction2 | |
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.streaming.InternalOutputModes | ||
| import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap | ||
| import org.apache.spark.sql.connector.catalog.{SupportsWrite, TableProvider} | ||
| import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableProvider} | ||
| import org.apache.spark.sql.connector.catalog.TableCapability._ | ||
| import org.apache.spark.sql.execution.command.DDLUtils | ||
| import org.apache.spark.sql.execution.datasources.DataSource | ||
|
|
@@ -45,6 +45,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap | |
| */ | ||
| @Evolving | ||
| final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { | ||
| import DataStreamWriter._ | ||
|
|
||
| private val df = ds.toDF() | ||
|
|
||
|
|
@@ -300,54 +301,55 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { | |
| "write files of Hive data source directly.") | ||
| } | ||
|
|
||
| if (source == "memory") { | ||
| assertNotPartitioned("memory") | ||
| if (source == SOURCE_NAME_TABLE) { | ||
| assertNotPartitioned(SOURCE_NAME_TABLE) | ||
|
|
||
| import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier | ||
|
|
||
| import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ | ||
| val originalMultipartIdentifier = df.sparkSession.sessionState.sqlParser | ||
| .parseMultipartIdentifier(tableName) | ||
| val CatalogAndIdentifier(catalog, identifier) = originalMultipartIdentifier | ||
|
|
||
| // Currently we don't create a logical streaming writer node in logical plan, so cannot rely | ||
| // on analyzer to resolve it. Directly lookup only for temp view to provide clearer message. | ||
| // TODO (SPARK-27484): we should add the writing node before the plan is analyzed. | ||
| if (df.sparkSession.sessionState.catalog.isTempView(originalMultipartIdentifier)) { | ||
| throw new AnalysisException(s"Temporary view $tableName doesn't support streaming write") | ||
| } | ||
|
|
||
| val tableInstance = catalog.asTableCatalog.loadTable(identifier) | ||
|
|
||
| import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ | ||
| val sink = tableInstance match { | ||
| case t: SupportsWrite if t.supports(STREAMING_WRITE) => t | ||
| case t => throw new AnalysisException(s"Table $tableName doesn't support streaming " + | ||
| s"write - $t") | ||
| } | ||
|
|
||
| startQuery(sink, extraOptions) | ||
| } else if (source == SOURCE_NAME_MEMORY) { | ||
| assertNotPartitioned(SOURCE_NAME_MEMORY) | ||
| if (extraOptions.get("queryName").isEmpty) { | ||
| throw new AnalysisException("queryName must be specified for memory sink") | ||
| } | ||
| val sink = new MemorySink() | ||
| val resultDf = Dataset.ofRows(df.sparkSession, new MemoryPlan(sink, df.schema.toAttributes)) | ||
| val chkpointLoc = extraOptions.get("checkpointLocation") | ||
| val recoverFromChkpoint = outputMode == OutputMode.Complete() | ||
| val query = df.sparkSession.sessionState.streamingQueryManager.startQuery( | ||
| extraOptions.get("queryName"), | ||
| chkpointLoc, | ||
| df, | ||
| extraOptions.toMap, | ||
| sink, | ||
| outputMode, | ||
| useTempCheckpointLocation = true, | ||
| recoverFromCheckpointLocation = recoverFromChkpoint, | ||
| trigger = trigger) | ||
| val query = startQuery(sink, extraOptions, recoverFromCheckpoint = recoverFromChkpoint) | ||
| resultDf.createOrReplaceTempView(query.name) | ||
| query | ||
| } else if (source == "foreach") { | ||
| assertNotPartitioned("foreach") | ||
| } else if (source == SOURCE_NAME_FOREACH) { | ||
| assertNotPartitioned(SOURCE_NAME_FOREACH) | ||
| val sink = ForeachWriterTable[T](foreachWriter, ds.exprEnc) | ||
| df.sparkSession.sessionState.streamingQueryManager.startQuery( | ||
| extraOptions.get("queryName"), | ||
| extraOptions.get("checkpointLocation"), | ||
| df, | ||
| extraOptions.toMap, | ||
| sink, | ||
| outputMode, | ||
| useTempCheckpointLocation = true, | ||
| trigger = trigger) | ||
| } else if (source == "foreachBatch") { | ||
| assertNotPartitioned("foreachBatch") | ||
| startQuery(sink, extraOptions) | ||
| } else if (source == SOURCE_NAME_FOREACH_BATCH) { | ||
| assertNotPartitioned(SOURCE_NAME_FOREACH_BATCH) | ||
| if (trigger.isInstanceOf[ContinuousTrigger]) { | ||
| throw new AnalysisException("'foreachBatch' is not supported with continuous trigger") | ||
| throw new AnalysisException(s"'$source' is not supported with continuous trigger") | ||
| } | ||
| val sink = new ForeachBatchSink[T](foreachBatchWriter, ds.exprEnc) | ||
| df.sparkSession.sessionState.streamingQueryManager.startQuery( | ||
| extraOptions.get("queryName"), | ||
| extraOptions.get("checkpointLocation"), | ||
| df, | ||
| extraOptions.toMap, | ||
| sink, | ||
| outputMode, | ||
| useTempCheckpointLocation = true, | ||
| trigger = trigger) | ||
| startQuery(sink, extraOptions) | ||
|
Member
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. @HeartSaVioR . For
Member
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. Previously, I and @cloud-fan hit case-sensitivity issues in another JIRAs due to this. Please make it sure that this PR doesn't re-introduce it because AS-IS PR switches
Member
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. If you already checked that, please add a test case for that. Or, we just use the old way
Contributor
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. Ah OK thanks for pointing out. Nice finding. I'll just explicitly call
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. nice catch @dongjoon-hyun ! |
||
| } else { | ||
| val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) | ||
| val disabledSources = df.sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",") | ||
|
|
@@ -380,19 +382,30 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { | |
| createV1Sink(optionsWithPath) | ||
| } | ||
|
|
||
| df.sparkSession.sessionState.streamingQueryManager.startQuery( | ||
| extraOptions.get("queryName"), | ||
| extraOptions.get("checkpointLocation"), | ||
| df, | ||
| optionsWithPath.originalMap, | ||
| sink, | ||
| outputMode, | ||
| useTempCheckpointLocation = source == "console" || source == "noop", | ||
| recoverFromCheckpointLocation = true, | ||
| trigger = trigger) | ||
| startQuery(sink, optionsWithPath.originalMap) | ||
| } | ||
| } | ||
|
|
||
| private def startQuery( | ||
| sink: Table, | ||
| newOptions: Map[String, String], | ||
| recoverFromCheckpoint: Boolean = true): StreamingQuery = { | ||
| val queryName = extraOptions.get("queryName") | ||
| val checkpointLocation = extraOptions.get("checkpointLocation") | ||
| val useTempCheckpointLocation = SOURCES_ALLOW_ONE_TIME_QUERY.contains(source) | ||
|
|
||
| df.sparkSession.sessionState.streamingQueryManager.startQuery( | ||
|
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. We can follow the previous code style
Contributor
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. OK let me keep it as it is. |
||
| queryName, | ||
| checkpointLocation, | ||
| df, | ||
| newOptions, | ||
| sink, | ||
| outputMode, | ||
| useTempCheckpointLocation = useTempCheckpointLocation, | ||
| recoverFromCheckpointLocation = recoverFromCheckpoint, | ||
| trigger = trigger) | ||
| } | ||
|
|
||
| private def createV1Sink(optionsWithPath: CaseInsensitiveMap[String]): Sink = { | ||
| val ds = DataSource( | ||
| df.sparkSession, | ||
|
|
@@ -409,7 +422,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { | |
| * @since 2.0.0 | ||
| */ | ||
| def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = { | ||
| this.source = "foreach" | ||
| this.source = SOURCE_NAME_FOREACH | ||
| this.foreachWriter = if (writer != null) { | ||
| ds.sparkSession.sparkContext.clean(writer) | ||
| } else { | ||
|
|
@@ -433,7 +446,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { | |
| */ | ||
| @Evolving | ||
| def foreachBatch(function: (Dataset[T], Long) => Unit): DataStreamWriter[T] = { | ||
| this.source = "foreachBatch" | ||
| this.source = SOURCE_NAME_FOREACH_BATCH | ||
| if (function == null) throw new IllegalArgumentException("foreachBatch function cannot be null") | ||
| this.foreachBatchWriter = function | ||
| this | ||
|
|
@@ -457,6 +470,17 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { | |
| foreachBatch((batchDs: Dataset[T], batchId: Long) => function.call(batchDs, batchId)) | ||
| } | ||
|
|
||
| /** | ||
| * Specifies the underlying output table. | ||
| * | ||
| * @since 3.1.0 | ||
| */ | ||
| def table(tableName: String): DataStreamWriter[T] = { | ||
|
||
| this.source = SOURCE_NAME_TABLE | ||
| this.tableName = tableName | ||
| this | ||
| } | ||
|
|
||
| private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map { cols => | ||
| cols.map(normalize(_, "Partition")) | ||
| } | ||
|
|
@@ -485,6 +509,8 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { | |
|
|
||
| private var source: String = df.sparkSession.sessionState.conf.defaultDataSourceName | ||
|
|
||
| private var tableName: String = null | ||
|
|
||
| private var outputMode: OutputMode = OutputMode.Append | ||
|
|
||
| private var trigger: Trigger = Trigger.ProcessingTime(0L) | ||
|
|
@@ -497,3 +523,16 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { | |
|
|
||
| private var partitioningColumns: Option[Seq[String]] = None | ||
| } | ||
|
|
||
| object DataStreamWriter { | ||
| val SOURCE_NAME_MEMORY = "memory" | ||
| val SOURCE_NAME_FOREACH = "foreach" | ||
| val SOURCE_NAME_FOREACH_BATCH = "foreachBatch" | ||
| val SOURCE_NAME_CONSOLE = "console" | ||
| val SOURCE_NAME_TABLE = "table" | ||
| val SOURCE_NAME_NOOP = "noop" | ||
|
|
||
| // these writer sources are also used for one-time query, hence allow temp checkpoint location | ||
| val SOURCES_ALLOW_ONE_TIME_QUERY = Seq(SOURCE_NAME_MEMORY, SOURCE_NAME_FOREACH, | ||
| SOURCE_NAME_FOREACH_BATCH, SOURCE_NAME_CONSOLE, SOURCE_NAME_NOOP) | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.