cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r491826956
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,54 @@ 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 (isTempView(df.sparkSession, 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)
- resultDf.createOrReplaceTempView(query.name)
- query
- } else if (source == "foreach") {
- assertNotPartitioned("foreach")
+ startQuery(sink, extraOptions, Some(resultDf), recoverFromCheckpoint =
recoverFromChkpoint)
+ } 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_NAME_FOREACH_BATCH' is not
supported with " +
Review comment:
nit: we can use `$source` to shorten the code.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -380,17 +381,44 @@ 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],
+ resultDf: Option[DataFrame] = None,
+ recoverFromCheckpoint: Boolean = true): StreamingQuery = {
+ val queryName = extraOptions.get("queryName")
+ val checkpointLocation = extraOptions.get("checkpointLocation")
+ val useTempCheckpointLocation =
SOURCES_ALLOW_ONE_TIME_QUERY.contains(source)
+
+ val query = df.sparkSession.sessionState.streamingQueryManager.startQuery(
+ queryName,
+ checkpointLocation,
+ df,
+ newOptions,
+ sink,
+ outputMode,
+ useTempCheckpointLocation = useTempCheckpointLocation,
+ recoverFromCheckpointLocation = recoverFromCheckpoint,
+ trigger = trigger)
+
+ resultDf.foreach { resDf => resDf.createOrReplaceTempView(query.name) }
Review comment:
only the in-memory sink passes in `resultDf`. Shall we register temp
view only in the `else if (source == "memory")` branch?
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -380,17 +381,44 @@ 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],
+ resultDf: Option[DataFrame] = None,
+ recoverFromCheckpoint: Boolean = true): StreamingQuery = {
+ val queryName = extraOptions.get("queryName")
+ val checkpointLocation = extraOptions.get("checkpointLocation")
+ val useTempCheckpointLocation =
SOURCES_ALLOW_ONE_TIME_QUERY.contains(source)
+
+ val query = df.sparkSession.sessionState.streamingQueryManager.startQuery(
+ queryName,
+ checkpointLocation,
+ df,
+ newOptions,
+ sink,
+ outputMode,
+ useTempCheckpointLocation = useTempCheckpointLocation,
+ recoverFromCheckpointLocation = recoverFromCheckpoint,
+ trigger = trigger)
+
+ resultDf.foreach { resDf => resDf.createOrReplaceTempView(query.name) }
+ query
+ }
+
+ private def isTempView(sparkSession: SparkSession, multiPartIdentifier:
Seq[String]): Boolean = {
+ val globalTempDBName = df.sparkSession.conf.get(
+ org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE)
+ val identifierForTempView = multiPartIdentifier match {
+ case Seq(dbName, tempViewName) if dbName.equals(globalTempDBName) =>
+ Seq(dbName, tempViewName)
+ case Seq(_, tempViewName) => Seq(tempViewName)
+ case ident => ident
}
+ df.sparkSession.sessionState.catalog.isTempView(identifierForTempView)
Review comment:
shall we simply call `...catalog.isTempView(multiPartIdentifier)`?
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]