xuanyuanking commented on a change in pull request #29756:
URL: https://github.com/apache/spark/pull/29756#discussion_r494027847



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
##########
@@ -260,19 +264,47 @@ class FindDataSourceTable(sparkSession: SparkSession) 
extends Rule[LogicalPlan]
     })
   }
 
+  private def getStreamingRelation(
+      table: CatalogTable,
+      extraOptions: CaseInsensitiveStringMap): StreamingRelation = {
+    val dsOptions = DataSourceUtils.generateDatasourceOptions(extraOptions, 
table)
+    val dataSource = DataSource(
+      sparkSession,
+      className = table.provider.get,
+      userSpecifiedSchema = Some(table.schema),
+      options = dsOptions)
+    StreamingRelation(dataSource)
+  }
+
+
   override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
-    case i @ InsertIntoStatement(UnresolvedCatalogRelation(tableMeta, 
options), _, _, _, _)
+    case i @ InsertIntoStatement(UnresolvedCatalogRelation(tableMeta, options, 
false), _, _, _, _)
         if DDLUtils.isDatasourceTable(tableMeta) =>
       i.copy(table = readDataSourceTable(tableMeta, options))
 
-    case i @ InsertIntoStatement(UnresolvedCatalogRelation(tableMeta, _), _, 
_, _, _) =>
+    case i @ InsertIntoStatement(UnresolvedCatalogRelation(tableMeta, _, 
false), _, _, _, _) =>
       i.copy(table = DDLUtils.readHiveTable(tableMeta))
 
-    case UnresolvedCatalogRelation(tableMeta, options) if 
DDLUtils.isDatasourceTable(tableMeta) =>
+    case UnresolvedCatalogRelation(tableMeta, options, false)
+        if DDLUtils.isDatasourceTable(tableMeta) =>
       readDataSourceTable(tableMeta, options)
 
-    case UnresolvedCatalogRelation(tableMeta, _) =>
+    case UnresolvedCatalogRelation(tableMeta, _, false) =>
       DDLUtils.readHiveTable(tableMeta)
+
+    case UnresolvedCatalogRelation(tableMeta, extraOptions, true) =>
+      getStreamingRelation(tableMeta, extraOptions)
+
+    case s @ StreamingRelationV2(
+        _, _, table, extraOptions, _, _, _, 
Some(UnresolvedCatalogRelation(tableMeta, _, true))) =>
+      import 
org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._
+      if (table.isInstanceOf[SupportsRead]
+          && table.supportsAny(MICRO_BATCH_READ, CONTINUOUS_READ)) {
+        s.copy(v1Relation = None)

Review comment:
       Yes, done in 97761d2

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala
##########
@@ -43,7 +43,8 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) {
       case r: DataSourceV2Relation if !r.table.supports(BATCH_READ) =>
         failAnalysis(s"Table ${r.table.name()} does not support batch scan.")
 
-      case r: StreamingRelationV2 if !r.table.supportsAny(MICRO_BATCH_READ, 
CONTINUOUS_READ) =>
+      case r: StreamingRelationV2
+          if !r.table.supportsAny(MICRO_BATCH_READ, CONTINUOUS_READ) && 
r.v1Relation.isEmpty =>

Review comment:
       Wrongly commit, reverted in 97761d2

##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.util
+
+import scala.collection.JavaConverters._
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, 
CatalogTable, CatalogTableType, V2TableWithV1Fallback}
+import org.apache.spark.sql.connector.{FakeV2Provider, InMemoryTableCatalog}
+import org.apache.spark.sql.connector.catalog.{Identifier, SupportsRead, 
Table, TableCapability}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.streaming.{MemoryStream, 
MemoryStreamScanBuilder, StreamingRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.StreamTest
+import org.apache.spark.sql.streaming.sources.FakeScanBuilder
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter {
+  import testImplicits._
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.testcat", 
classOf[InMemoryTableCatalog].getName)
+    spark.conf.set("spark.sql.catalog.teststream", 
classOf[InMemoryStreamTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.clear()
+  }
+
+  test("table API with file source") {
+    Seq("parquet", "").foreach { source =>
+      withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> source) {
+        withTempDir { tempDir =>
+          val tblName = "my_table"
+          val dir = tempDir.getAbsolutePath
+          withTable(tblName) {
+            spark.range(3).write.format("parquet").option("path", 
dir).saveAsTable(tblName)
+
+            testStream(spark.readStream.table(tblName))(
+              ProcessAllAvailable(),
+              CheckAnswer(Row(0), Row(1), Row(2))
+            )
+          }
+        }
+      }
+    }
+  }
+
+  test("read non-exist table") {
+    intercept[AnalysisException] {
+      spark.readStream.table("non_exist_table")
+    }.message.contains("Table not found")
+  }
+
+  test("stream table API with temp view") {
+    val tblName = "my_table"
+    val stream = MemoryStream[Int]
+    withTable(tblName) {
+      stream.toDF().createOrReplaceTempView(tblName)
+
+      testStream(spark.readStream.table(tblName)) (
+        AddData(stream, 1, 2, 3),
+        CheckLastBatch(1, 2, 3),
+        AddData(stream, 4, 5),
+        CheckLastBatch(4, 5)
+      )
+    }
+  }
+
+  test("stream table API with non-streaming temp view") {
+    val tblName = "my_table"
+    withTable(tblName) {
+      spark.range(3).createOrReplaceTempView(tblName)
+      intercept[AnalysisException] {
+        spark.readStream.table(tblName)
+      }.message.contains("not supported")
+    }
+  }
+
+  test("read table without streaming capability support") {
+    val tableIdentifer = "testcat.table_name"
+
+    spark.sql(s"CREATE TABLE $tableIdentifer (id bigint, data string) USING 
foo")
+
+    intercept[AnalysisException] {
+      spark.readStream.table(tableIdentifer)
+    }.message.contains("does not support either micro-batch or continuous 
scan")
+  }
+
+  test("read table with custom catalog") {
+    val tblName = "teststream.table_name"
+    withTable(tblName) {
+      spark.sql(s"CREATE TABLE $tblName (data int) USING foo")
+      val stream = MemoryStream[Int]
+      val testCatalog = 
spark.sessionState.catalogManager.catalog("teststream").asTableCatalog
+      val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
+      table.asInstanceOf[InMemoryStreamTable].setStream(stream)
+
+      testStream(spark.readStream.table(tblName)) (
+        AddData(stream, 1, 2, 3),
+        CheckLastBatch(1, 2, 3),
+        AddData(stream, 4, 5),
+        CheckLastBatch(4, 5)
+      )
+    }
+  }
+
+  test("read table with custom catalog & namespace") {
+    spark.sql("CREATE NAMESPACE teststream.ns")
+
+    val tblName = "teststream.ns.table_name"
+    withTable(tblName) {
+      spark.sql(s"CREATE TABLE $tblName (data int) USING foo")
+      val stream = MemoryStream[Int]
+      val testCatalog = 
spark.sessionState.catalogManager.catalog("teststream").asTableCatalog
+      val table = testCatalog.loadTable(Identifier.of(Array("ns"), 
"table_name"))
+      table.asInstanceOf[InMemoryStreamTable].setStream(stream)
+
+      testStream(spark.readStream.table(tblName)) (
+        AddData(stream, 1, 2, 3),
+        CheckLastBatch(1, 2, 3),
+        AddData(stream, 4, 5),
+        CheckLastBatch(4, 5)
+      )
+    }
+  }
+
+  test("fallback to V1 relation") {
+    val tblName = DataStreamTableAPISuite.V1FallbackTestTableName
+    spark.conf.set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key,
+      classOf[InMemoryStreamTableCatalog].getName)
+    val v2Source = classOf[FakeV2Provider].getName
+    withTempDir { tempDir =>
+      withTable(tblName) {
+        spark.sql(s"CREATE TABLE $tblName (data int) USING $v2Source")
+        val plan = spark.readStream.option("path", 
tempDir.getCanonicalPath).table(tblName)
+          .queryExecution.analyzed.collectFirst {
+          case d: StreamingRelation => d

Review comment:
       Make sense, done in 97761d2




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



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

Reply via email to