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/16 03:13:30 UTC

[GitHub] [spark] HeartSaVioR opened a new pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

HeartSaVioR opened a new pull request #29767:
URL: https://github.com/apache/spark/pull/29767


   ### What changes were proposed in this pull request?
   
   This PR proposes to add `DataStreamWriter.table` to specify the output "table" to write from the streaming query.
   
   ### Why are the changes needed?
   
   For now, there's no way to write to the table (especially catalog table) even the table is capable to handle streaming write, so even with Spark 3, writing to the catalog table via SS should go through the `DataStreamWriter.format(provider)` and wish the provider can handle it as same as we do with catalog table.
   
   With the new API, we can directly point to the catalog table which supports streaming write. Some of usages are covered with tests - simply saying, end users can do the following:
   
   ```
   // assuming `testcat` is a custom catalog, and `ns` is a namespace in the catalog
   spark.sql("CREATE TABLE testcat.ns.table1 (id bigint, data string) USING foo")
   
   val query = inputDF
         .writeStream
         .table("testcat.ns.table1")
         .option(...)
         .start()
   ```
   
   ### Does this PR introduce _any_ user-facing change?
   
   Yes, as this adds a new public API in DataStreamWriter. This doesn't bring backward incompatible change.
   
   ### How was this patch tested?
   
   New unit tests.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-701897135


   **[Test build #129302 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129302/testReport)** for PR 29767 at commit [`7d5ea52`](https://github.com/apache/spark/commit/7d5ea52800dbfe9f117b9bcb5fdc839f811fefbc).


----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705352048






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693146454






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490662661



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala
##########
@@ -169,26 +171,32 @@ class InMemoryTable(
 
     new WriteBuilder with SupportsTruncate with SupportsOverwrite with SupportsDynamicOverwrite {
       private var writer: BatchWrite = Append
+      private var streamingWriter: StreamingWrite = StreamingAppend
 
       override def truncate(): WriteBuilder = {
         assert(writer == Append)
         writer = TruncateAndAppend
+        streamingWriter = StreamingTruncateAndAppend
         this
       }
 
       override def overwrite(filters: Array[Filter]): WriteBuilder = {
         assert(writer == Append)
         writer = new Overwrite(filters)
+        // streaming writer doesn't have equivalent semantic

Review comment:
       Yes, at least for now. (If I understand correctly.) If we would like to be sure, we may be able to assign dummy one and throw error on calling `buildForStreaming()`. Probably it'd be much clearer.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r491827924



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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693226706


   **[Test build #128751 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128751/testReport)** for PR 29767 at commit [`f557696`](https://github.com/apache/spark/commit/f5576969ee9afeaa35c3bb74822aad43cf9c11c2).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501236088



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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)

Review comment:
       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 `extraOptions.toMap` -> `extraOptions` silently.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705396162


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34153/
   


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694122566


   **[Test build #128814 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128814/testReport)** for PR 29767 at commit [`1179c2f`](https://github.com/apache/spark/commit/1179c2f15b7461dec513fb679416ac5f4488d9a1).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695434433






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490721265



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       I think we should fix `DataFrameWriterV2` as well, to fail if the table name refers to a temp view. cc @rdblue 




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-706917712


   Just update the title is good enough, it's only about the commit message


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705512232


   **[Test build #129548 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129548/testReport)** for PR 29767 at commit [`3f80c4f`](https://github.com/apache/spark/commit/3f80c4fc78ca4fcfc1f6a843a3175b403fb4bb3c).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704211353


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129441/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694702308


   **[Test build #128865 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128865/testReport)** for PR 29767 at commit [`e99a0d7`](https://github.com/apache/spark/commit/e99a0d7dcc2724d63c6ebae0898c0f8f796a0eeb).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694702903






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705376287


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694122566


   **[Test build #128814 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128814/testReport)** for PR 29767 at commit [`1179c2f`](https://github.com/apache/spark/commit/1179c2f15b7461dec513fb679416ac5f4488d9a1).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694610552






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693373524


   **[Test build #128767 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128767/testReport)** for PR 29767 at commit [`f557696`](https://github.com/apache/spark/commit/f5576969ee9afeaa35c3bb74822aad43cf9c11c2).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704392076


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34059/
   


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705405292


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34153/
   


----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695906232


   Shall we have another round of review? Thanks in advance!


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704221808


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704238785


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34050/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501450993



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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)

Review comment:
       Ah OK thanks for pointing out. Nice finding. I'll just explicitly call `.toMap` as it was.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694664243


   **[Test build #128855 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128855/testReport)** for PR 29767 at commit [`e7cd27d`](https://github.com/apache/spark/commit/e7cd27dd3afd32d9b28e13d56af26f6b5e097c85).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501221080



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
##########
@@ -169,6 +171,147 @@ class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter {
       }
     }
   }
+
+  test("write: write to table with custom catalog & no namespace") {
+    val tableIdentifier = "testcat.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to table with custom catalog & namespace") {
+    spark.sql("CREATE NAMESPACE testcat.ns")
+
+    val tableIdentifier = "testcat.ns.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to table with default session catalog") {
+    val v2Source = classOf[FakeV2Provider].getName
+    spark.conf.set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key,
+      classOf[InMemoryTableSessionCatalog].getName)
+
+    spark.sql("CREATE NAMESPACE ns")
+
+    val tableIdentifier = "ns.table_name"
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING $v2Source")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to non-exist table with custom catalog") {
+    val tableIdentifier = "testcat.nonexisttable"
+    spark.sql("CREATE NAMESPACE testcat.ns")
+
+    withTempDir { checkpointDir =>
+      val exc = intercept[NoSuchTableException] {
+        runStreamQueryAppendMode(tableIdentifier, checkpointDir, Seq.empty, Seq.empty)
+      }
+      assert(exc.getMessage.contains("nonexisttable"))
+    }
+  }
+
+  test("write: write to file provider based table isn't allowed yet") {
+    val tableIdentifier = "table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING parquet")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    withTempDir { checkpointDir =>
+      val exc = intercept[AnalysisException] {
+        runStreamQueryAppendMode(tableIdentifier, checkpointDir, Seq.empty, Seq.empty)
+      }
+      assert(exc.getMessage.contains("doesn't support streaming write"))
+    }
+  }
+
+  test("write: write to temporary view isn't allowed yet") {

Review comment:
       Thank you for adding this explicitly.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705376287






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705513024






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693370596






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695434433






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693145074






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490190733



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       I'm playing a bit more with view, and unlike temporary view it seems to be loaded via loadTable. Now checking capability.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490190573



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+      val tableInstance = catalog.asTableCatalog.loadTable(identifier)

Review comment:
       Ah sorry for the unclear, I mean the behavior when the table does not exist. Should we support create a new table and append data into? Seems a good-to-have feature. IMO it's the significant difference between the reader and the writer side.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694123178






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705353239






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705415349






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694701965


   retest this please


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490074969



##########
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") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.{NonSessionCatalogAndIdentifier, CatalogAndIdentifier}
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val tableInstance = df.sparkSession.sessionState.sqlParser
+        .parseMultipartIdentifier(tableName) match {
+
+        case NonSessionCatalogAndIdentifier(catalog, ident) =>

Review comment:
       this is not needed anymore.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693352950


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128751/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501450163



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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] = {

Review comment:
       OK I'll change the name to saveAsTable and call start() there.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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)

Review comment:
       Ah OK thanks for pointing out. Nice finding. I'll just explicitly call `.toMap` as it was.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -380,19 +396,31 @@ 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)
     }
   }
 
+  private def startQuery(
+      sink: Table,
+      newOptions: CaseInsensitiveMap[String],
+      recoverFromCheckpoint: Boolean = true): StreamingQuery = {
+    val options = newOptions.originalMap
+    val queryName = options.get("queryName")

Review comment:
       OK I'll let it as it is.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -380,19 +396,31 @@ 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)
     }
   }
 
+  private def startQuery(
+      sink: Table,
+      newOptions: CaseInsensitiveMap[String],
+      recoverFromCheckpoint: Boolean = true): StreamingQuery = {
+    val options = newOptions.originalMap
+    val queryName = options.get("queryName")
+    val checkpointLocation = options.get("checkpointLocation")
+    val useTempCheckpointLocation = SOURCES_ALLOW_ONE_TIME_QUERY.contains(source)
+
+    df.sparkSession.sessionState.streamingQueryManager.startQuery(

Review comment:
       OK let me keep it as it is.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490651024



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+      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("Table doesn't support streaming " +

Review comment:
       s"Table $tableName doesn't support streaming "?




----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-706912198


   You're right. Would you like to go with revert & another PR, or it's just for information? Either is fine for me.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693352946






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693370596






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693974559


   **[Test build #128801 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128801/testReport)** for PR 29767 at commit [`6444a1e`](https://github.com/apache/spark/commit/6444a1e85dac18c368158ebdc60b91ec6a727b06).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704207549


   **[Test build #129441 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129441/testReport)** for PR 29767 at commit [`7e7aff3`](https://github.com/apache/spark/commit/7e7aff3d8aa9f483855123c5f81c4b395a6f10fd).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705376587


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129547/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704238256


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34048/
   


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695751809






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704392110






----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705377157


   retest this, please


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r500199076



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
##########
@@ -169,6 +171,150 @@ class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter {
       }
     }
   }
+
+  test("write: write to table with custom catalog & no namespace") {
+    val tableIdentifier = "testcat.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to table with custom catalog & namespace") {
+    spark.sql("CREATE NAMESPACE testcat.ns")
+
+    val tableIdentifier = "testcat.ns.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to table with default session catalog") {
+    val v2Source = classOf[FakeV2Provider].getName
+    spark.conf.set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key,
+      classOf[InMemoryTableSessionCatalog].getName)
+
+    spark.sql("CREATE NAMESPACE ns")
+
+    val tableIdentifier = "ns.table_name"
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING $v2Source")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to non-exist table with custom catalog") {
+    val tableIdentifier = "testcat.nonexisttable"
+    val existTableIdentifier = "testcat.ns.nonexisttable"
+
+    spark.sql("CREATE NAMESPACE testcat.ns")
+    spark.sql(s"CREATE TABLE $existTableIdentifier (id bigint, data string) USING foo")
+
+    withTempDir { checkpointDir =>
+      val exc = intercept[NoSuchTableException] {
+        runStreamQueryAppendMode(tableIdentifier, checkpointDir, Seq.empty, Seq.empty)
+      }
+      assert(exc.getMessage.contains("nonexisttable"))
+    }
+  }
+
+  test("write: write to file provider based table isn't allowed yet") {

Review comment:
       The tests I added are more likely E2E tests so that we don't miss something. A table backed by file source is definitely something we want to ensure whether it's working or not at specific moment. When we support streaming write on file source v2, this test should be modified at that time to verify it.
   
   If we want to also add a test for V2 table which doesn't support streaming, it warrants a new test. Please let me know whether we'd like to add it.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490645075



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       For temporary view, this change makes the test work:
   
   ```
       spark.table(tableIdentifier).createOrReplaceTempView(tempViewIdentifier)
       // or
       // spark.read.table(tableIdentifier).createOrReplaceTempView(tempViewIdentifier)
       Seq((1, "a"), (2, "b"), (3, "c")).toDF().write.insertInto(tempViewIdentifier)
   ```
   
   but I'm not sure about the coverage - it sounds to me that temp view as just an alias of the table is only supported for insertInto.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695947893






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490214083



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       Looks like it requires handling V1Table after loadTable (for view), as well as pattern match with `AsTableIdentifier(tableIdentifier)` (for temporary view).
   
   In either way, I see DataFrameWriter leverages UnresolvedRelation to defer resolution, but streaming query doesn't add a writer node in logical plan and passes the actual table instance (either SupportsWrite for V2 or Sink for V1) directly, so the situation looks to be a bit different.
   
   (Btw, interesting one to test even on batch query. Probably I'd test with creating temp view with V2 table and try to write. If that would work for DataFrameWriter.insertInto, that's probably one thing which DataFrameWriterV2 may not support as of now, as it doesn't have fail-back to V1 path.)




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490715412



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       Done in e7cd27d - now it looks up (global) temp view directly and provide error message a bit clearer. Also added relevant tests.
   
   That said, I can't find the logic for fail-back in DataFrameWriterV2. It simply looks up with catalog, which temp view will not be found. Do I understand correctly, and if then is it a desired/expected behavior?




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694332515






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r492412994



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       Filed SPARK-32960 and submitted a PR. (#29830) Please take a look and let me know whether it follows your suggestion properly or not. Thanks!




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705415330


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34154/
   


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693146454


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r497990865



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamWriterWithTableSuite.scala
##########
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.streaming.test
+
+import java.io.File
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
+import org.apache.spark.sql.connector.{FakeV2Provider, InMemoryTableCatalog, InMemoryTableSessionCatalog}
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION
+import org.apache.spark.sql.streaming.StreamTest
+
+class DataStreamWriterWithTableSuite extends StreamTest with BeforeAndAfter {

Review comment:
       Done in 7d5ea52




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695751664


   **[Test build #128900 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128900/testReport)** for PR 29767 at commit [`e99a0d7`](https://github.com/apache/spark/commit/e99a0d7dcc2724d63c6ebae0898c0f8f796a0eeb).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693148547






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704217946


   **[Test build #129443 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129443/testReport)** for PR 29767 at commit [`e051fb9`](https://github.com/apache/spark/commit/e051fb9205c3220983d88b6c6a928429236aa627).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490043221



##########
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") {
+    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) =>

Review comment:
       It's OK. `V2SessionCatalog.loadTable` checks the namespace as well.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704321439


   The build should be fixed by #29955


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704392110






----------------------------------------------------------------
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


[GitHub] [spark] dongjoon-hyun closed pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun closed pull request #29767:
URL: https://github.com/apache/spark/pull/29767


   


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705370899


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34149/
   


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694089909






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r500188785



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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] = {

Review comment:
       I have a bit different view on DataStreamWriter (and probably DataFrameWriter as well):
   
   While we don't restrict the order, actually I think it's pretty much natural to have a flow, like `define a sink` -> `set options to the sink` -> `set options to the streaming query` -> `start the query`. (A couple of parts can be consolidated or the sequence can be swapped.)
   
   ```
   df.writeStream
      .format("...")
      .option("...")
      .outputMode(...)
      .trigger(...)
      .start()
   ```
   
   Now it looks to be simply arbitrary and something got mixed up. `checkpointLocation` isn't something being tied to the sink but we let end users to put into `option` which is also used for sink. `queryName` as well.
   
   I intended the addition of `table` method as `defining a sink`, but if we'd like to care for tables specially, `DataFrameWriter.insertInto` would match the intention and I can change the method name to `insertInto` here as well. WDYT?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r500084205



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
##########
@@ -169,6 +171,150 @@ class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter {
       }
     }
   }
+
+  test("write: write to table with custom catalog & no namespace") {
+    val tableIdentifier = "testcat.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to table with custom catalog & namespace") {
+    spark.sql("CREATE NAMESPACE testcat.ns")
+
+    val tableIdentifier = "testcat.ns.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to table with default session catalog") {
+    val v2Source = classOf[FakeV2Provider].getName
+    spark.conf.set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key,
+      classOf[InMemoryTableSessionCatalog].getName)
+
+    spark.sql("CREATE NAMESPACE ns")
+
+    val tableIdentifier = "ns.table_name"
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING $v2Source")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to non-exist table with custom catalog") {
+    val tableIdentifier = "testcat.nonexisttable"
+    val existTableIdentifier = "testcat.ns.nonexisttable"
+
+    spark.sql("CREATE NAMESPACE testcat.ns")
+    spark.sql(s"CREATE TABLE $existTableIdentifier (id bigint, data string) USING foo")
+
+    withTempDir { checkpointDir =>
+      val exc = intercept[NoSuchTableException] {
+        runStreamQueryAppendMode(tableIdentifier, checkpointDir, Seq.empty, Seq.empty)
+      }
+      assert(exc.getMessage.contains("nonexisttable"))
+    }
+  }
+
+  test("write: write to file provider based table isn't allowed yet") {

Review comment:
       can we use a fake v2 provider that doesn't support streaming? We might support file source v2 streaming write in the future.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694089308


   **[Test build #128809 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128809/testReport)** for PR 29767 at commit [`6c040f3`](https://github.com/apache/spark/commit/6c040f376fc86b0b28d079595ca96facdc91a1fc).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r496727460



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamWriterWithTableSuite.scala
##########
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.streaming.test
+
+import java.io.File
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
+import org.apache.spark.sql.connector.{FakeV2Provider, InMemoryTableCatalog, InMemoryTableSessionCatalog}
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION
+import org.apache.spark.sql.streaming.StreamTest
+
+class DataStreamWriterWithTableSuite extends StreamTest with BeforeAndAfter {

Review comment:
       Can we merge this suite into `DataStreamTableAPISuite`?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r500081260



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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] = {

Review comment:
       It's a bit weird to define `table` as a config method. I think it's better to follow `DataFrameWriter.saveAsTable` and make it an action. How about
   ```
   def table(tableName: String): StreamingQuery = {
     this.source = SOURCE_NAME_TABLE
     this.tableName = tableName
     start()
   }
   ```




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490662104



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+      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("Table doesn't support streaming " +
+          s"write - $t")
+      }
+
+      startQuery(sink, extraOptions)
+    } else 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
+      startQuery(sink, extraOptions, Some(resultDf), recoverFromCheckpoint = recoverFromChkpoint)
     } 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)
+      startQuery(sink, extraOptions)
     } else if (source == "foreachBatch") {

Review comment:
       Ah I missed after doing another approach of refactoring. I'll fix. Thanks!




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694664677






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-706883233


   We need to update the PR title, it's saveAsTable not table.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704527936






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693146444


   **[Test build #128736 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128736/testReport)** for PR 29767 at commit [`bfba28b`](https://github.com/apache/spark/commit/bfba28b5e0d5b94d7ddc034f996959a175fcbcca).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693225169


   retest this, please


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693352946


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705382669


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34149/
   


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694025779






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695432759


   **[Test build #128900 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128900/testReport)** for PR 29767 at commit [`e99a0d7`](https://github.com/apache/spark/commit/e99a0d7dcc2724d63c6ebae0898c0f8f796a0eeb).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694610751






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705405311






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704380750


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34059/
   


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705405311






----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705352048


   Thanks for reviewing. Addressed review comments. Please take a look again.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694269567






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501476018



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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)

Review comment:
       nice catch @dongjoon-hyun !

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -380,19 +396,31 @@ 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)
     }
   }
 
+  private def startQuery(
+      sink: Table,
+      newOptions: CaseInsensitiveMap[String],
+      recoverFromCheckpoint: Boolean = true): StreamingQuery = {
+    val options = newOptions.originalMap
+    val queryName = options.get("queryName")

Review comment:
       previously it was `extraOptions.get("queryName")`, we should follow it to get the `queryName` option case insensitively.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -380,19 +396,31 @@ 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)
     }
   }
 
+  private def startQuery(
+      sink: Table,
+      newOptions: CaseInsensitiveMap[String],
+      recoverFromCheckpoint: Boolean = true): StreamingQuery = {
+    val options = newOptions.originalMap
+    val queryName = options.get("queryName")
+    val checkpointLocation = options.get("checkpointLocation")

Review comment:
       ditto

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -380,19 +396,31 @@ 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)
     }
   }
 
+  private def startQuery(
+      sink: Table,
+      newOptions: CaseInsensitiveMap[String],
+      recoverFromCheckpoint: Boolean = true): StreamingQuery = {
+    val options = newOptions.originalMap
+    val queryName = options.get("queryName")
+    val checkpointLocation = options.get("checkpointLocation")
+    val useTempCheckpointLocation = SOURCES_ALLOW_ONE_TIME_QUERY.contains(source)
+
+    df.sparkSession.sessionState.streamingQueryManager.startQuery(

Review comment:
       We can follow the previous code style
   ```
   ...startQuery(
     newOptions.get("queryName"),
     newOptions.get("checkpointLocation"),
     df,
     newOptions. originalMap,
     ...
   ```




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694685690






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695947372


   **[Test build #128930 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128930/testReport)** for PR 29767 at commit [`b980431`](https://github.com/apache/spark/commit/b980431409255f7b81aff1d2fa81bc2871dff630).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-700713022


   Let's get this in first. 


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704211345


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r492412994



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       Filed SPARK-32960 and submitted a PR. (#29830) Please take a look and let me know whether it follows your suggestion properly or not. Thanks!




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704221808






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694696483






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705353239






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705513024


   Merged build finished. Test PASSed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693227405






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490724544



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,53 @@ 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 CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+
+      // 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, identifier.asMultipartIdentifier)) {

Review comment:
       Please correct me if I'm missing here. The reason I pass all parts in identifier here is to cover global temp view, which uses global temp db. Dropping the db name (if it isn't from global temp db) is performed in `isTempView`.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-701919260


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33917/
   


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694267213


   **[Test build #128809 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128809/testReport)** for PR 29767 at commit [`6c040f3`](https://github.com/apache/spark/commit/6c040f376fc86b0b28d079595ca96facdc91a1fc).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695947893






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r489137110



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
##########
@@ -815,3 +818,131 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter {
     }
   }
 }
+
+class DataStreamWriterWithTableSuite extends StreamTest with BeforeAndAfter {
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.clear()
+    sqlContext.streams.active.foreach(_.stop())
+  }
+
+  test("write to table with custom catalog & no namespace") {
+    val tableIdentifier = "testcat.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write to table with custom catalog & namespace") {
+    spark.sql("CREATE NAMESPACE testcat.ns")
+
+    val tableIdentifier = "testcat.ns.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write to table with default session catalog") {
+    try {
+      val v2Source = classOf[FakeV2Provider].getName
+      spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION.key,
+        classOf[InMemoryTableSessionCatalog].getName)
+
+      spark.sql("CREATE NAMESPACE ns")
+
+      val tableIdentifier = "ns.table_name"
+      spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING $v2Source")
+      checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+      runTestWithStreamAppend(tableIdentifier)
+    } finally {
+      spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key)
+    }
+  }
+
+  test("write to non-exist table with custom catalog") {
+    val tableIdentifier = "testcat.nonexisttable"
+    val existTableIdentifier = "testcat.ns.nonexisttable"
+
+    spark.sql("CREATE NAMESPACE testcat.ns")
+    spark.sql(s"CREATE TABLE $existTableIdentifier (id bigint, data string) USING foo")
+
+    withTempDir { checkpointDir =>
+      val exc = intercept[NoSuchTableException] {
+        runStreamQueryAppendMode(tableIdentifier, checkpointDir, Seq.empty, Seq.empty)
+      }
+      assert(exc.getMessage.contains("nonexisttable"))
+    }
+  }
+
+  test("write to file provider based table shouldn't be allowed yet") {
+    val tableIdentifier = "table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING parquet")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    withTempDir { checkpointDir =>
+      val exc = intercept[AnalysisException] {

Review comment:
       This is because file provider based table is V1 which doesn't have capability of streaming write. I hope this is OK, rather than struggling to convert it into Sink and making it work anyway.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501476820



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -380,19 +396,31 @@ 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)
     }
   }
 
+  private def startQuery(
+      sink: Table,
+      newOptions: CaseInsensitiveMap[String],
+      recoverFromCheckpoint: Boolean = true): StreamingQuery = {
+    val options = newOptions.originalMap
+    val queryName = options.get("queryName")

Review comment:
       previously it was `extraOptions.get("queryName")`, we should follow it to get the `queryName` option case insensitively.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -380,19 +396,31 @@ 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)
     }
   }
 
+  private def startQuery(
+      sink: Table,
+      newOptions: CaseInsensitiveMap[String],
+      recoverFromCheckpoint: Boolean = true): StreamingQuery = {
+    val options = newOptions.originalMap
+    val queryName = options.get("queryName")
+    val checkpointLocation = options.get("checkpointLocation")

Review comment:
       ditto




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694691078






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693351886


   **[Test build #128751 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128751/testReport)** for PR 29767 at commit [`f557696`](https://github.com/apache/spark/commit/f5576969ee9afeaa35c3bb74822aad43cf9c11c2).
    * This patch **fails PySpark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694630939






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694696547






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704526462


   **[Test build #129452 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129452/testReport)** for PR 29767 at commit [`e051fb9`](https://github.com/apache/spark/commit/e051fb9205c3220983d88b6c6a928429236aa627).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693216742


   **[Test build #128739 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128739/testReport)** for PR 29767 at commit [`f557696`](https://github.com/apache/spark/commit/f5576969ee9afeaa35c3bb74822aad43cf9c11c2).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490178331



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       I just checked it roughly, and looks like temporary view is not loaded by `loadTable` - it throws NoSuchTableException in V2SessionCatalog.
   
   ```
   
     test("write to temporary view shouldn't be allowed") {
       val tableIdentifier = "table_name"
       val tempViewIdentifier = "temp_view"
   
       spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING parquet")
       checkAnswer(spark.table(tableIdentifier), Seq.empty)
   
       spark.sql(s"SELECT id, data FROM $tableIdentifier").createOrReplaceTempView(tempViewIdentifier)
   
       // spark.sql(s"CREATE TEMPORARY VIEW $tempViewIdentifier AS SELECT id, data FROM $tableIdentifier")
   
       withTempDir { checkpointDir =>
         val exc = intercept[AnalysisException] {
           runStreamQueryAppendMode("default." + tempViewIdentifier, checkpointDir, Seq.empty, Seq.empty)
         }
         assert(exc.getMessage.contains("doesn't support streaming write"))
       }
     }
   ```
   
   fails with `"Table default.temp_view not found;" did not contain "doesn't support streaming write"`
   




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693226706


   **[Test build #128751 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128751/testReport)** for PR 29767 at commit [`f557696`](https://github.com/apache/spark/commit/f5576969ee9afeaa35c3bb74822aad43cf9c11c2).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693148256


   **[Test build #128739 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128739/testReport)** for PR 29767 at commit [`f557696`](https://github.com/apache/spark/commit/f5576969ee9afeaa35c3bb74822aad43cf9c11c2).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705379129


   **[Test build #129548 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129548/testReport)** for PR 29767 at commit [`3f80c4f`](https://github.com/apache/spark/commit/3f80c4fc78ca4fcfc1f6a843a3175b403fb4bb3c).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490176830



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+      val tableInstance = catalog.asTableCatalog.loadTable(identifier)

Review comment:
       Could you please elaborate? Which one do we want to handle here except DSv2 table?
   (Personally I would say we should accelerate migration to DSv2 instead of spending efforts to extend supporting DSv1.) 




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501237095



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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)

Review comment:
       If you already checked that, please add a test case for that. Or, we just use the old way `extraOptions.toMap` to avoid any side effect.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694006865






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694630669


   **[Test build #128850 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128850/testReport)** for PR 29767 at commit [`4fd5cb9`](https://github.com/apache/spark/commit/4fd5cb9586c9a2badd4025082f811829c60bade6).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r500188785



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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] = {

Review comment:
       I have a bit different view on DataStreamWriter (and probably DataFrameWriter as well):
   
   While we don't restrict the order, actually I think it's pretty much natural to have a flow, like `define a sink` -> `set options to the sink` -> `set options to the streaming query` -> `start the query`. (A couple of parts can be consolidated or the sequence can be swapped.)
   
   ```
   df.writeStream
      .format("...")
      .option("...")
      .outputMode(...)
      .trigger(...)
      .start()
   ```
   
   Now it looks to be simply arbitrary and something got mixed up. `checkpointLocation` isn't something being tied to the sink but we let end users to put into `option` which is also used for sink. `queryName` as well.
   
   I intended the addition of `table` method as `defining a sink`, but if we'd like to care for tables specially, `DataFrameWriter.insertInto` would match the intention and I can change the method name to `insertInto` here as well. (I'm also fine to add it as `saveAsTable`.)
   
   WDYT?




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704221818


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129443/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693217408


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128739/
   Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490683314



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala
##########
@@ -169,26 +171,32 @@ class InMemoryTable(
 
     new WriteBuilder with SupportsTruncate with SupportsOverwrite with SupportsDynamicOverwrite {
       private var writer: BatchWrite = Append
+      private var streamingWriter: StreamingWrite = StreamingAppend
 
       override def truncate(): WriteBuilder = {
         assert(writer == Append)
         writer = TruncateAndAppend
+        streamingWriter = StreamingTruncateAndAppend
         this
       }
 
       override def overwrite(filters: Array[Filter]): WriteBuilder = {
         assert(writer == Append)
         writer = new Overwrite(filters)
+        // streaming writer doesn't have equivalent semantic

Review comment:
       Just changed to explicitly fail for the case.




----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693700523


   cc. @cloud-fan @tdas @zsxwing @gaborgsomogyi @xuanyuanking 


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693975766






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705382678






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694855886






----------------------------------------------------------------
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


[GitHub] [spark] dongjoon-hyun commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695428325


   Retest this please


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694089909






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694024365


   **[Test build #128804 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128804/testReport)** for PR 29767 at commit [`7d4f6a1`](https://github.com/apache/spark/commit/7d4f6a1170ba074671ddb6a2104cf89973aa1d76).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490003646



##########
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") {
+    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) =>

Review comment:
       shall we just use `CatalogAndIdentifier`?




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704238284






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r492412994



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       Filed SPARK-32960 and submitted a PR. (#29830) Please take a look and let me know whether it follows your suggestion properly or not. Thanks!




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693145074






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693217397






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705376287






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694006697


   **[Test build #128801 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128801/testReport)** for PR 29767 at commit [`6444a1e`](https://github.com/apache/spark/commit/6444a1e85dac18c368158ebdc60b91ec6a727b06).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490721856



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,53 @@ 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 CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+
+      // 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, identifier.asMultipartIdentifier)) {

Review comment:
       This is incorrect. The `identifier` is for a specific catalog. e.g.  for `cat1.t1`, the `identifier` is `t1`.  `cat1.t1` is not a temp view, but `t1` might be.
   
   We should check temp view with the original table name.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704207549


   **[Test build #129441 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129441/testReport)** for PR 29767 at commit [`7e7aff3`](https://github.com/apache/spark/commit/7e7aff3d8aa9f483855123c5f81c4b395a6f10fd).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501239376



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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] = {

Review comment:
       then the API works differently between `DataFrameWriter` and `DataStreamWriter`, and users may be confusing.
   
   I'm fine with `saveAsTable` if it's more like action.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-702067890






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693975766






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490730813



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,53 @@ 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 CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+
+      // 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, identifier.asMultipartIdentifier)) {

Review comment:
       My bad. Thanks for explaining. I see the failing case when catalog "exists" for the head of identifier; let me fix it immediately.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490151791



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+      val tableInstance = catalog.asTableCatalog.loadTable(identifier)

Review comment:
       Skip for the table non-existence is good for the first version, but it seems a good-to-have one in the future.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       Is it possible to get a temp view here and what the behavior should be?




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694664243


   **[Test build #128855 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128855/testReport)** for PR 29767 at commit [`e7cd27d`](https://github.com/apache/spark/commit/e7cd27dd3afd32d9b28e13d56af26f6b5e097c85).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501479016



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -380,19 +396,31 @@ 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)
     }
   }
 
+  private def startQuery(
+      sink: Table,
+      newOptions: CaseInsensitiveMap[String],
+      recoverFromCheckpoint: Boolean = true): StreamingQuery = {
+    val options = newOptions.originalMap
+    val queryName = options.get("queryName")
+    val checkpointLocation = options.get("checkpointLocation")
+    val useTempCheckpointLocation = SOURCES_ALLOW_ONE_TIME_QUERY.contains(source)
+
+    df.sparkSession.sessionState.streamingQueryManager.startQuery(

Review comment:
       OK let me keep it as it is.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694198420






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694697204


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128850/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705353239


   **[Test build #129544 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129544/testReport)** for PR 29767 at commit [`716a615`](https://github.com/apache/spark/commit/716a61525da01385c271ae8dc2a8b410717401b6).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705376295






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501476018



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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)

Review comment:
       nice catch @dongjoon-hyun !




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-702066375


   **[Test build #129302 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129302/testReport)** for PR 29767 at commit [`7d5ea52`](https://github.com/apache/spark/commit/7d5ea52800dbfe9f117b9bcb5fdc839f811fefbc).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695947372






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694269567






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r491827460



##########
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?




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693373524


   **[Test build #128767 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128767/testReport)** for PR 29767 at commit [`f557696`](https://github.com/apache/spark/commit/f5576969ee9afeaa35c3bb74822aad43cf9c11c2).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704229344


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34048/
   


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694006884


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128801/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704250822






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704211345






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693144673


   **[Test build #128736 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128736/testReport)** for PR 29767 at commit [`bfba28b`](https://github.com/apache/spark/commit/bfba28b5e0d5b94d7ddc034f996959a175fcbcca).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694198420






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490662661



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala
##########
@@ -169,26 +171,32 @@ class InMemoryTable(
 
     new WriteBuilder with SupportsTruncate with SupportsOverwrite with SupportsDynamicOverwrite {
       private var writer: BatchWrite = Append
+      private var streamingWriter: StreamingWrite = StreamingAppend
 
       override def truncate(): WriteBuilder = {
         assert(writer == Append)
         writer = TruncateAndAppend
+        streamingWriter = StreamingTruncateAndAppend
         this
       }
 
       override def overwrite(filters: Array[Filter]): WriteBuilder = {
         assert(writer == Append)
         writer = new Overwrite(filters)
+        // streaming writer doesn't have equivalent semantic

Review comment:
       Yes, at least for now. If we would like to be sure, we may be able to assign dummy one and throw error on calling `buildForStreaming()`. Probably it'd be much clearer.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694702308


   **[Test build #128865 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128865/testReport)** for PR 29767 at commit [`e99a0d7`](https://github.com/apache/spark/commit/e99a0d7dcc2724d63c6ebae0898c0f8f796a0eeb).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-696065314


   **[Test build #128930 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128930/testReport)** for PR 29767 at commit [`b980431`](https://github.com/apache/spark/commit/b980431409255f7b81aff1d2fa81bc2871dff630).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501450163



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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] = {

Review comment:
       OK I'll change the name to saveAsTable and call start() there.




----------------------------------------------------------------
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


[GitHub] [spark] dongjoon-hyun closed pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun closed pull request #29767:
URL: https://github.com/apache/spark/pull/29767


   


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705379129


   **[Test build #129548 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129548/testReport)** for PR 29767 at commit [`3f80c4f`](https://github.com/apache/spark/commit/3f80c4fc78ca4fcfc1f6a843a3175b403fb4bb3c).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705376287






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694025779






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694664677






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694610552


   **[Test build #128844 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128844/testReport)** for PR 29767 at commit [`1197e0b`](https://github.com/apache/spark/commit/1197e0bd11b23923c1afb1a6200728b470c4bc85).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-701919277






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
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.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695947372


   **[Test build #128930 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128930/testReport)** for PR 29767 at commit [`b980431`](https://github.com/apache/spark/commit/b980431409255f7b81aff1d2fa81bc2871dff630).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693148256


   **[Test build #128739 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128739/testReport)** for PR 29767 at commit [`f557696`](https://github.com/apache/spark/commit/f5576969ee9afeaa35c3bb74822aad43cf9c11c2).


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-701909616


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33917/
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501234564



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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)

Review comment:
       @HeartSaVioR . For `CaseInsensitiveMap`, `def toMap: Map[String, T] = originalMap`. It seems that we need `toMap` explicitly here as we did line 385. 
   ```
   startQuery(sink, optionsWithPath.originalMap)
   ```

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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)

Review comment:
       @HeartSaVioR . For `CaseInsensitiveMap`, `def toMap: Map[String, T] = originalMap`. It seems that we need `toMap` explicitly here as we did line 385. (cc @cloud-fan )
   ```
   startQuery(sink, optionsWithPath.originalMap)
   ```




----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704267920


   The build failure doesn't look to occur from my change, though that looks to be warn but considered as error and build finally failed.
   
   ```
   [error] [warn] /home/jenkins/workspace/SparkPullRequestBuilder/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala:129: This catches all Throwables. If this is really intended, use `case e : Throwable` to clear this warning.
   [error] [warn]         case e => throw e
   [error] [warn] 
   ```
   
   Did we change something on build like parameters/options?


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r500275109



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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] = {

Review comment:
       I think we will have `DataStreamWriterV2` eventually (after we figure out how to design output mode). For now, it's more important to keep API consistency between batch and stream.
   
   I don't have a strong opinion about the naming, `table` is fine. cc @xuanyuanking @zsxwing 




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694690671


   **[Test build #128855 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128855/testReport)** for PR 29767 at commit [`e7cd27d`](https://github.com/apache/spark/commit/e7cd27dd3afd32d9b28e13d56af26f6b5e097c85).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r489137859



##########
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")

Review comment:
       The huge diff comes from refactor - I have to go with refactoring as the statement for StreamingQueryManager.startQuery() are all duplicated and I was about to add one more duplication.
   
   The actual change is only performed for `source == SOURCE_NAME_TABLE`.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693597505


   **[Test build #128767 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128767/testReport)** for PR 29767 at commit [`f557696`](https://github.com/apache/spark/commit/f5576969ee9afeaa35c3bb74822aad43cf9c11c2).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693217397


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694123178






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704217946


   **[Test build #129443 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129443/testReport)** for PR 29767 at commit [`e051fb9`](https://github.com/apache/spark/commit/e051fb9205c3220983d88b6c6a928429236aa627).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r500284785



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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] = {

Review comment:
       IMHO the exact match of the method from batch side looks to be `insertInto` (as it will handle the output mode and simply add/update the data instead of creating a table). Just naming it to `table` doesn't look to say it's an action - in DataFrameWriter it's `saveAsTable` instead of simply `table`.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501236088



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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)

Review comment:
       Previously, I and @cloud-fan hits case-sensitivity issues due to this. Please make it sure that this PR doesn't re-introduce it because AS-IS PR switches `extraOptions.toMap` -> `extraOptions` silently.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704238284






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694702903






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490739744



##########
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)

Review comment:
       shall we make it a parameter and ask the caller side to pass in?




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705376011






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694006865


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695947893






----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.saveAsTable API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-706931429


   Ah OK. Thanks for the guidance. I've updated the PR title and description as well, as the usage is a bit different from before.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705513028


   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/129548/
   Test PASSed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490645075



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       For temporary view, this change makes the test work:
   
   ```
       spark.table(tableIdentifier).createOrReplaceTempView(tempViewIdentifier)
       Seq((1, "a"), (2, "b"), (3, "c")).toDF().write.insertInto(tempViewIdentifier)
   ```
   
   but I'm not sure about the coverage - it sounds to me that temp view as just an alias of the table is only supported for insertInto.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490652352



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+      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("Table doesn't support streaming " +
+          s"write - $t")
+      }
+
+      startQuery(sink, extraOptions)
+    } else 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
+      startQuery(sink, extraOptions, Some(resultDf), recoverFromCheckpoint = recoverFromChkpoint)
     } else if (source == "foreach") {

Review comment:
       SOURCE_NAME_FOREACH?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+      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("Table doesn't support streaming " +
+          s"write - $t")
+      }
+
+      startQuery(sink, extraOptions)
+    } else if (source == "memory") {

Review comment:
       SOURCE_NAME_MEMORY?




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-696066094






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693148547






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694630939






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490652267



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+      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("Table doesn't support streaming " +
+          s"write - $t")
+      }
+
+      startQuery(sink, extraOptions)
+    } else 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
+      startQuery(sink, extraOptions, Some(resultDf), recoverFromCheckpoint = recoverFromChkpoint)
     } 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)
+      startQuery(sink, extraOptions)
     } else if (source == "foreachBatch") {

Review comment:
       SOURCE_NAME_FOREACH_BATCH?




----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693369939


   retest this, please


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490696111



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       > (only if the temp view is a single data source scan node)
   
   As I mentioned before, the temp view must be very simple, like `spark.table(name)` or `CREATE TEMP VIEW v USING parquet OPTIONS(...)`
   
   I believe there are tests, but I don't remember where they are. You can update `ResolveRelations` to drop the support of inserting temp views, and see which tests fail.
   
   For this particular PR, I'm OK to not support temp view for now, as we need to refactor it a little bit and have a logical plan for streaming write. But for consistency with other places that lookup a table, we should still lookup temp views, and just fail if a temp view is returned.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705405694


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34154/
   


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704221791


   **[Test build #129443 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129443/testReport)** for PR 29767 at commit [`e051fb9`](https://github.com/apache/spark/commit/e051fb9205c3220983d88b6c6a928429236aa627).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-697178139


   Shall we continue review this (prior to #29830) so that the change can be used on the tests for #29756 ? Thanks in advance.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705376570






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704211325


   **[Test build #129441 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129441/testReport)** for PR 29767 at commit [`7e7aff3`](https://github.com/apache/spark/commit/7e7aff3d8aa9f483855123c5f81c4b395a6f10fd).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694855886






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490214083



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       Looks like it requires handling V1Table after loadTable (for view), as well as pattern match with `AsTableIdentifier(tableIdentifier)` (for temporary view).
   
   In either way, I see DataFrameWriter leverages UnresolvedRelation to defer resolution, but streaming query doesn't add a writer node in logical plan and passes the actual table instance (either SupportsWrite for V2 or Sink for V1) directly, so the situation looks to be a bit different. Probably another reason to add writer node before analyzing?
   
   (Btw, interesting one to test even on batch query. Probably I'd test with creating temp view with V2 table and try to write. If that would work for DataFrameWriter.insertInto, that's probably one thing which DataFrameWriterV2 may not support as of now, as it doesn't have fail-back to V1 path.)




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694685690






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490178331



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       I just checked it roughly, and looks like temporary view is not loaded by `loadTable` - it throws NoSuchTableException in V2SessionCatalog.
   
   ```
   
     test("write to temporary view shouldn't be allowed") {
       val tableIdentifier = "table_name"
       val tempViewIdentifier = "temp_view"
   
       spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING parquet")
       checkAnswer(spark.table(tableIdentifier), Seq.empty)
   
       spark.sql(s"SELECT id, data FROM $tableIdentifier").createOrReplaceTempView(tempViewIdentifier)
   
       // spark.sql(s"CREATE TEMPORARY VIEW $tempViewIdentifier AS SELECT id, data FROM $tableIdentifier")
   
       withTempDir { checkpointDir =>
         val exc = intercept[AnalysisException] {
           runStreamQueryAppendMode("default." + tempViewIdentifier, checkpointDir, Seq.empty, Seq.empty)
         }
         assert(exc.getMessage.contains("doesn't support streaming write"))
       }
     }
   ```
   
   fails with `"Table default.temp_view not found;" did not contain "doesn't support streaming write"`.
   
   For sure I think this is desired behavior, as it's a view. Even it can load the (temp) view, capability shouldn't have write related flags.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490642895



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser

Review comment:
       @cloud-fan 
   Does the functionality (DataFrameWriter.insertInto with (temp) view) being covered by tests?
   
   ```
     test("insertInto to view against file-based table") {
       val tableIdentifier = "table_name"
       val viewIdentifier = "table_view"
   
       spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING parquet")
       checkAnswer(spark.table(tableIdentifier), Seq.empty)
   
       spark.sql(s"CREATE VIEW $viewIdentifier AS SELECT id, data FROM $tableIdentifier")
   
       spark.range(10).write.insertInto(viewIdentifier)
   
       /*
       Inserting into a view is not allowed. View: `default`.`table_view`.;
       org.apache.spark.sql.AnalysisException: Inserting into a view is not allowed. View: `default`.`table_view`.;
        */
     }
   
     test("insertInto to temp view against file-based table") {
       val tableIdentifier = "table_name"
       val tempViewIdentifier = "temp_view"
   
       spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING parquet")
       checkAnswer(spark.table(tableIdentifier), Seq.empty)
   
       spark.sql(s"SELECT id, data FROM $tableIdentifier").createOrReplaceTempView(tempViewIdentifier)
   
       spark.range(10).write.insertInto(tempViewIdentifier)
   
       /*
       Inserting into an RDD-based table is not allowed.;;
       'InsertIntoStatement Project [id#0L, data#1], false, false
       +- Range (0, 10, step=1, splits=Some(2))
   
       org.apache.spark.sql.AnalysisException: Inserting into an RDD-based table is not allowed.;;
       'InsertIntoStatement Project [id#0L, data#1], false, false
       +- Range (0, 10, step=1, splits=Some(2))
   
       --> Actually the relation is under the projection
       Project [id#0L, data#1]
       +- Relation[id#0L,data#1] parquet
        */
     }
   
     test("insertInto to view against V2 table") {
       val tableIdentifier = "testcat.table_name"
       val viewIdentifier = "table_view"
   
       spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
       checkAnswer(spark.table(tableIdentifier), Seq.empty)
   
       spark.sql(s"CREATE VIEW $viewIdentifier AS SELECT id, data FROM $tableIdentifier")
   
       spark.range(10).write.insertInto(viewIdentifier)
   
       /*
       Inserting into a view is not allowed. View: `default`.`table_view`.;
       org.apache.spark.sql.AnalysisException: Inserting into a view is not allowed. View: `default`.`table_view`.;
        */
     }
   
     test("insertInto to temp view against V2 table") {
       val tableIdentifier = "testcat.table_name"
       val tempViewIdentifier = "temp_view"
   
       spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
       checkAnswer(spark.table(tableIdentifier), Seq.empty)
   
       spark.sql(s"SELECT id, data FROM $tableIdentifier").createOrReplaceTempView(tempViewIdentifier)
   
       spark.range(10).write.insertInto(tempViewIdentifier)
   
       /*
       Inserting into an RDD-based table is not allowed.;;
       'InsertIntoStatement Project [id#8L, data#9], false, false
       +- Range (0, 10, step=1, splits=Some(2))
   
       org.apache.spark.sql.AnalysisException: Inserting into an RDD-based table is not allowed.;;
       'InsertIntoStatement Project [id#8L, data#9], false, false
       +- Range (0, 10, step=1, splits=Some(2))
   
       --> Actually the relation is under the projection
       Project [id#8L, data#9]
       +- RelationV2[id#8L, data#9] testcat.table_name
        */
     }
   ```
   
   All tests failed.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705353239






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r501229623



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -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] = {

Review comment:
       Ya. I also agree with the AS-IS config method `table` instead of action.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490730813



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,53 @@ 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 CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+
+      // 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, identifier.asMultipartIdentifier)) {

Review comment:
       Thanks for explaining. I see the failing case when catalog "exists" for the head of identifier; let me fix it immediately.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490006162



##########
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") {
+    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) =>

Review comment:
       Is it OK to skip the namespace length check in SessionCatalogAndIdentifier here?




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694330736


   **[Test build #128814 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128814/testReport)** for PR 29767 at commit [`1179c2f`](https://github.com/apache/spark/commit/1179c2f15b7461dec513fb679416ac5f4488d9a1).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704250822


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r489991853



##########
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:
       I'm OK on the refactored result in either way. I'll try it and see the code diff. Thanks!




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490006671



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
##########
@@ -815,3 +818,131 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter {
     }
   }
 }
+
+class DataStreamWriterWithTableSuite extends StreamTest with BeforeAndAfter {
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.clear()
+    sqlContext.streams.active.foreach(_.stop())
+  }
+
+  test("write to table with custom catalog & no namespace") {
+    val tableIdentifier = "testcat.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write to table with custom catalog & namespace") {
+    spark.sql("CREATE NAMESPACE testcat.ns")
+
+    val tableIdentifier = "testcat.ns.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write to table with default session catalog") {
+    try {
+      val v2Source = classOf[FakeV2Provider].getName
+      spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION.key,
+        classOf[InMemoryTableSessionCatalog].getName)
+
+      spark.sql("CREATE NAMESPACE ns")
+
+      val tableIdentifier = "ns.table_name"
+      spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING $v2Source")
+      checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+      runTestWithStreamAppend(tableIdentifier)
+    } finally {
+      spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key)
+    }
+  }
+
+  test("write to non-exist table with custom catalog") {
+    val tableIdentifier = "testcat.nonexisttable"
+    val existTableIdentifier = "testcat.ns.nonexisttable"
+
+    spark.sql("CREATE NAMESPACE testcat.ns")
+    spark.sql(s"CREATE TABLE $existTableIdentifier (id bigint, data string) USING foo")
+
+    withTempDir { checkpointDir =>
+      val exc = intercept[NoSuchTableException] {
+        runStreamQueryAppendMode(tableIdentifier, checkpointDir, Seq.empty, Seq.empty)
+      }
+      assert(exc.getMessage.contains("nonexisttable"))
+    }
+  }
+
+  test("write to file provider based table shouldn't be allowed yet") {
+    val tableIdentifier = "table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING parquet")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    withTempDir { checkpointDir =>
+      val exc = intercept[AnalysisException] {

Review comment:
       This is a new API. I'm OK with not supporting streaming v1 sink at first.




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694332515






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490201003



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,44 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
         "write files of Hive data source directly.")
     }
 
-    if (source == "memory") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier
+
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+      val CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+      val tableInstance = catalog.asTableCatalog.loadTable(identifier)

Review comment:
       Ah OK. It brings additional options as we see in DataFrameWriterV2 to support create table, so if we want to do that, I think it warrants DataStreamWriterV2.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490076987



##########
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") {
+    if (source == SOURCE_NAME_TABLE) {
+      assertNotPartitioned("table")
+
+      import df.sparkSession.sessionState.analyzer.{NonSessionCatalogAndIdentifier, CatalogAndIdentifier}
+
+      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 CatalogAndIdentifier(catalog, ident) =>
+          catalog.asTableCatalog.loadTable(ident)
+
+        case other =>

Review comment:
       See `DataFrameWriterV2.scala#L52`, we can simply write
   ```
   val CatalogAndIdentifier(catalog, identifier) = ...parseMultipartIdentifier(tableName)
   val table = catalog.asTableCatalog.loadTable(identifier)
   ```




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705365907


   **[Test build #129547 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129547/testReport)** for PR 29767 at commit [`3f80c4f`](https://github.com/apache/spark/commit/3f80c4fc78ca4fcfc1f6a843a3175b403fb4bb3c).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r500083684



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
##########
@@ -169,6 +171,150 @@ class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter {
       }
     }
   }
+
+  test("write: write to table with custom catalog & no namespace") {
+    val tableIdentifier = "testcat.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to table with custom catalog & namespace") {
+    spark.sql("CREATE NAMESPACE testcat.ns")
+
+    val tableIdentifier = "testcat.ns.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to table with default session catalog") {
+    val v2Source = classOf[FakeV2Provider].getName
+    spark.conf.set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key,
+      classOf[InMemoryTableSessionCatalog].getName)
+
+    spark.sql("CREATE NAMESPACE ns")
+
+    val tableIdentifier = "ns.table_name"
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING $v2Source")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write: write to non-exist table with custom catalog") {
+    val tableIdentifier = "testcat.nonexisttable"
+    val existTableIdentifier = "testcat.ns.nonexisttable"
+
+    spark.sql("CREATE NAMESPACE testcat.ns")
+    spark.sql(s"CREATE TABLE $existTableIdentifier (id bigint, data string) USING foo")

Review comment:
       why we create a table but not use it?




----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694697057






----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-706912198


   You're right. Would you like to go with revert & another PR, or it's just for information?


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-705415349






----------------------------------------------------------------
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


[GitHub] [spark] SparkQA removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704326247


   **[Test build #129452 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129452/testReport)** for PR 29767 at commit [`e051fb9`](https://github.com/apache/spark/commit/e051fb9205c3220983d88b6c6a928429236aa627).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693227405






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-701919277






----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490005962



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
##########
@@ -815,3 +818,131 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter {
     }
   }
 }
+
+class DataStreamWriterWithTableSuite extends StreamTest with BeforeAndAfter {
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.clear()
+    sqlContext.streams.active.foreach(_.stop())
+  }
+
+  test("write to table with custom catalog & no namespace") {
+    val tableIdentifier = "testcat.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write to table with custom catalog & namespace") {
+    spark.sql("CREATE NAMESPACE testcat.ns")
+
+    val tableIdentifier = "testcat.ns.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING foo")
+    checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+    runTestWithStreamAppend(tableIdentifier)
+  }
+
+  test("write to table with default session catalog") {
+    try {
+      val v2Source = classOf[FakeV2Provider].getName
+      spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION.key,
+        classOf[InMemoryTableSessionCatalog].getName)
+
+      spark.sql("CREATE NAMESPACE ns")
+
+      val tableIdentifier = "ns.table_name"
+      spark.sql(s"CREATE TABLE $tableIdentifier (id bigint, data string) USING $v2Source")
+      checkAnswer(spark.table(tableIdentifier), Seq.empty)
+
+      runTestWithStreamAppend(tableIdentifier)
+    } finally {
+      spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key)

Review comment:
       In `after` we clear out everything, so this is not needed.




----------------------------------------------------------------
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


[GitHub] [spark] HeartSaVioR commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695906232


   Shall we have another round of review? Thanks in advance!


----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694089308


   **[Test build #128809 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128809/testReport)** for PR 29767 at commit [`6c040f3`](https://github.com/apache/spark/commit/6c040f376fc86b0b28d079595ca96facdc91a1fc).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490004721



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala
##########
@@ -815,3 +818,131 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter {
     }
   }
 }
+
+class DataStreamWriterWithTableSuite extends StreamTest with BeforeAndAfter {

Review comment:
       shall we move it to a new file?




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29767:
URL: https://github.com/apache/spark/pull/29767#discussion_r490726820



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
##########
@@ -300,54 +301,53 @@ 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 CatalogAndIdentifier(catalog, identifier) = df.sparkSession.sessionState.sqlParser
+          .parseMultipartIdentifier(tableName)
+
+      // 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, identifier.asMultipartIdentifier)) {

Review comment:
       >  I pass all parts in identifier
   
   But you did not... The catalog name is missing, so you may mistakenly treat a table as temp view, e.g. `cat1.t1` if `t1` is the name of a temp view.




----------------------------------------------------------------
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


[GitHub] [spark] SparkQA commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-704326247


   **[Test build #129452 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129452/testReport)** for PR 29767 at commit [`e051fb9`](https://github.com/apache/spark/commit/e051fb9205c3220983d88b6c6a928429236aa627).


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-695947893






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693146457


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128736/
   Test FAILed.


----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-694610751






----------------------------------------------------------------
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


[GitHub] [spark] AmplabJenkins commented on pull request #29767: [SPARK-32896][SS] Add DataStreamWriter.table API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29767:
URL: https://github.com/apache/spark/pull/29767#issuecomment-693599799






----------------------------------------------------------------
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