You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/09/17 05:57:10 UTC

[GitHub] [spark] cloud-fan commented on a change in pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r489988109



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,97 +301,108 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
-      assertNotPartitioned("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")
-      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")
-      if (trigger.isInstanceOf[ContinuousTrigger]) {
-        throw new AnalysisException("'foreachBatch' 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)
-    } else {
-      val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf)
-      val disabledSources = df.sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",")
-      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 queryName = extraOptions.get("queryName")
+    val checkpointLocation = extraOptions.get("checkpointLocation")
+    val useTempCheckpointLocation = SOURCES_ALLOW_ONE_TIME_QUERY.contains(source)
+
+    val (sink, resultDf, recoverFromCheckpoint, newOptions) = {
+      if (source == SOURCE_NAME_TABLE) {
+        assertNotPartitioned("table")
+
+        import df.sparkSession.sessionState.analyzer.{NonSessionCatalogAndIdentifier, SessionCatalogAndIdentifier}
+
+        import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+        val tableInstance = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName) match {
+
+          case NonSessionCatalogAndIdentifier(catalog, ident) =>
+            catalog.asTableCatalog.loadTable(ident)
+
+          case SessionCatalogAndIdentifier(catalog, ident) =>
+            catalog.asTableCatalog.loadTable(ident)
+
+          case other =>
+            throw new AnalysisException(
+              s"Couldn't find a catalog to handle the identifier ${other.quoted}.")
+        }
 
-      val sink = if (classOf[TableProvider].isAssignableFrom(cls) && !useV1Source) {
-        val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider]
-        val sessionOptions = DataSourceV2Utils.extractSessionConfigs(
-          source = provider, conf = df.sparkSession.sessionState.conf)
-        val finalOptions = sessionOptions.filterKeys(!optionsWithPath.contains(_)).toMap ++
-          optionsWithPath.originalMap
-        val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava)
-        val table = DataSourceV2Utils.getTableFromProvider(
-          provider, dsOptions, userSpecifiedSchema = None)
         import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._
-        table match {
-          case table: SupportsWrite if table.supports(STREAMING_WRITE) =>
-            table
-          case _ => createV1Sink(optionsWithPath)
+        val sink = tableInstance match {
+          case t: SupportsWrite if t.supports(STREAMING_WRITE) => t
+          case t => throw new AnalysisException("Table doesn't support streaming " +
+            s"write - $t")
+        }
+
+        (sink, None, true, extraOptions)
+      } else if (source == SOURCE_NAME_MEMORY) {
+        assertNotPartitioned("memory")
+        if (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))
+
+        (sink, Some(resultDf), outputMode == OutputMode.Complete(), extraOptions)
+      } else if (source == SOURCE_NAME_FOREACH) {
+        assertNotPartitioned("foreach")
+        val sink = ForeachWriterTable[T](foreachWriter, ds.exprEnc)
+        (sink, None, true, extraOptions)
+      } else if (source == SOURCE_NAME_FOREACH_BATCH) {
+        assertNotPartitioned("foreachBatch")
+        if (trigger.isInstanceOf[ContinuousTrigger]) {
+          throw new AnalysisException("'foreachBatch' is not supported with continuous trigger")
         }
+        val sink = new ForeachBatchSink[T](foreachBatchWriter, ds.exprEnc)
+        (sink, None, true, extraOptions)
       } else {
-        createV1Sink(optionsWithPath)
-      }
+        val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf)
+        val disabledSources = df.sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",")
+        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) {
+          val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider]
+          val sessionOptions = DataSourceV2Utils.extractSessionConfigs(
+            source = provider, conf = df.sparkSession.sessionState.conf)
+          val finalOptions = sessionOptions.filterKeys(!optionsWithPath.contains(_)).toMap ++
+            optionsWithPath.originalMap
+          val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava)
+          val table = DataSourceV2Utils.getTableFromProvider(
+            provider, dsOptions, userSpecifiedSchema = None)
+          import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._
+          table match {
+            case table: SupportsWrite if table.supports(STREAMING_WRITE) =>
+              table
+            case _ => createV1Sink(optionsWithPath)
+          }
+        } else {
+          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)
+        (sink, None, true, optionsWithPath.originalMap)
+      }
     }
+
+    val query = df.sparkSession.sessionState.streamingQueryManager.startQuery(

Review comment:
       another way to refactor:
   ```
   private def startQuery(sink, resultDf, recoverFromCheckpoint, newOptions) = {
     df.sparkSession.sessionState.streamingQueryManager.startQuery...
   }
   
   def start() = {
     if (source == SOURCE_NAME_TABLE) = {
       ...
       startQuery...
     } else if (...) {
       ...
       startQuery...
     } ...
   }
   ```
   
   this should reduce code diff quite a bit.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org