[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-25 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -861,15 +862,22 @@ class Analyzer(
 lookupTempView(ident).map(_ => 
ResolvedView(ident.asIdentifier)).getOrElse(u)
 }
 
-def lookupTempView(identifier: Seq[String]): Option[LogicalPlan] = {
+def lookupTempView(
+identifier: Seq[String], isStreaming: Boolean = false): 
Option[LogicalPlan] = {
   // Permanent View can't refer to temp views, no need to lookup at all.
   if (isResolvingView) return None
 
-  identifier match {
+  val tmpView = identifier match {
 case Seq(part1) => v1SessionCatalog.lookupTempView(part1)
 case Seq(part1, part2) => v1SessionCatalog.lookupGlobalTempView(part1, 
part2)
 case _ => None
   }
+
+  if (isStreaming && tmpView.nonEmpty && !tmpView.get.isStreaming) {
+throw new AnalysisException("The temp view related to non-streaming 
relation is " +

Review comment:
   ```
   "${identifier.quoted} is a temp view of batch logical plan, please use batch 
API such as `DataFrameReader.table` to read it."
   ```

##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -861,15 +862,22 @@ class Analyzer(
 lookupTempView(ident).map(_ => 
ResolvedView(ident.asIdentifier)).getOrElse(u)
 }
 
-def lookupTempView(identifier: Seq[String]): Option[LogicalPlan] = {
+def lookupTempView(
+identifier: Seq[String], isStreaming: Boolean = false): 
Option[LogicalPlan] = {
   // Permanent View can't refer to temp views, no need to lookup at all.
   if (isResolvingView) return None
 
-  identifier match {
+  val tmpView = identifier match {
 case Seq(part1) => v1SessionCatalog.lookupTempView(part1)
 case Seq(part1, part2) => v1SessionCatalog.lookupGlobalTempView(part1, 
part2)
 case _ => None
   }
+
+  if (isStreaming && tmpView.nonEmpty && !tmpView.get.isStreaming) {
+throw new AnalysisException("The temp view related to non-streaming 
relation is " +

Review comment:
   ```
   "${identifier.quoted} is not a temp view of streaming logical plan, please 
use batch API such as `DataFrameReader.table` to read it."
   ```

##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -1020,16 +1039,38 @@ class Analyzer(
 // 3) If a v1 table is found, create a v1 relation. Otherwise, create a v2 
relation.
 private def lookupRelation(
 identifier: Seq[String],
-options: CaseInsensitiveStringMap): Option[LogicalPlan] = {
+options: CaseInsensitiveStringMap,
+isStreaming: Boolean): Option[LogicalPlan] = {
   expandRelationName(identifier) match {
 case SessionCatalogAndIdentifier(catalog, ident) =>
   lazy val loaded = CatalogV2Util.loadTable(catalog, ident).map {
 case v1Table: V1Table =>
-  v1SessionCatalog.getRelation(v1Table.v1Table, options)
+  if (isStreaming) {
+if (v1Table.v1Table.tableType == CatalogTableType.VIEW) {
+  throw new AnalysisException("Stream reading does not support 
views.")

Review comment:
   ```
   "${identifier.quoted} is a permanent view, which is not supported by 
streaming reading API such as `DataStreamReader.table` yet."
   ```

##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
##
@@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeMap, Attri
 import org.apache.spark.sql.catalyst.plans.logical._
 import 
org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils
 import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.connector.catalog.{CatalogManager, Table}

Review comment:
   unnecessary change?

##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala
##
@@ -80,3 +80,11 @@ private[sql] case class V1Table(v1Table: CatalogTable) 
extends Table {
 
   override def toString: String = s"V1Table($name)"
 }
+
+/**
+ * A V2 table with V1 fallback support. This is used to fallback to V1 table 
when the V2 one
+ * doesn't implement specific capabilities but V1 already has.
+ */
+trait V2TableWithV1Fallback extends Table {

Review comment:
   please add `private[sql]`





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:

[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-24 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala
##
@@ -80,3 +80,11 @@ private[sql] case class V1Table(v1Table: CatalogTable) 
extends Table {
 
   override def toString: String = s"V1Table($name)"
 }
+
+/**
+ * A V2 table with V1 fallback support. This is used to fallback to V1 table 
when the V2 one
+ * doesn't implement specific capabilities but V1 already has.
+ */
+trait V2TableWithV1Fallback extends Table {

Review comment:
   please add `private[sql]`





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-24 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -1020,16 +1039,38 @@ class Analyzer(
 // 3) If a v1 table is found, create a v1 relation. Otherwise, create a v2 
relation.
 private def lookupRelation(
 identifier: Seq[String],
-options: CaseInsensitiveStringMap): Option[LogicalPlan] = {
+options: CaseInsensitiveStringMap,
+isStreaming: Boolean): Option[LogicalPlan] = {
   expandRelationName(identifier) match {
 case SessionCatalogAndIdentifier(catalog, ident) =>
   lazy val loaded = CatalogV2Util.loadTable(catalog, ident).map {
 case v1Table: V1Table =>
-  v1SessionCatalog.getRelation(v1Table.v1Table, options)
+  if (isStreaming) {
+if (v1Table.v1Table.tableType == CatalogTableType.VIEW) {
+  throw new AnalysisException("Stream reading does not support 
views.")

Review comment:
   ```
   "${identifier.quoted} is a permanent view, which is not supported by 
streaming reading API such as `DataStreamReader.table` yet."
   ```





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-24 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
##
@@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeMap, Attri
 import org.apache.spark.sql.catalyst.plans.logical._
 import 
org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils
 import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.connector.catalog.{CatalogManager, Table}

Review comment:
   unnecessary change?





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-24 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -861,15 +862,22 @@ class Analyzer(
 lookupTempView(ident).map(_ => 
ResolvedView(ident.asIdentifier)).getOrElse(u)
 }
 
-def lookupTempView(identifier: Seq[String]): Option[LogicalPlan] = {
+def lookupTempView(
+identifier: Seq[String], isStreaming: Boolean = false): 
Option[LogicalPlan] = {
   // Permanent View can't refer to temp views, no need to lookup at all.
   if (isResolvingView) return None
 
-  identifier match {
+  val tmpView = identifier match {
 case Seq(part1) => v1SessionCatalog.lookupTempView(part1)
 case Seq(part1, part2) => v1SessionCatalog.lookupGlobalTempView(part1, 
part2)
 case _ => None
   }
+
+  if (isStreaming && tmpView.nonEmpty && !tmpView.get.isStreaming) {
+throw new AnalysisException("The temp view related to non-streaming 
relation is " +

Review comment:
   ```
   "${identifier.quoted} is not a temp view of streaming logical plan, please 
use batch API such as `DataFrameReader.table` to read 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:
us...@infra.apache.org



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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-24 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -861,15 +862,22 @@ class Analyzer(
 lookupTempView(ident).map(_ => 
ResolvedView(ident.asIdentifier)).getOrElse(u)
 }
 
-def lookupTempView(identifier: Seq[String]): Option[LogicalPlan] = {
+def lookupTempView(
+identifier: Seq[String], isStreaming: Boolean = false): 
Option[LogicalPlan] = {
   // Permanent View can't refer to temp views, no need to lookup at all.
   if (isResolvingView) return None
 
-  identifier match {
+  val tmpView = identifier match {
 case Seq(part1) => v1SessionCatalog.lookupTempView(part1)
 case Seq(part1, part2) => v1SessionCatalog.lookupGlobalTempView(part1, 
part2)
 case _ => None
   }
+
+  if (isStreaming && tmpView.nonEmpty && !tmpView.get.isStreaming) {
+throw new AnalysisException("The temp view related to non-streaming 
relation is " +

Review comment:
   ```
   "${identifier.quoted} is a temp view of batch logical plan, please use batch 
API such as `DataFrameReader.table` to read 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:
us...@infra.apache.org



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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-23 Thread GitBox


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



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

[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-23 Thread GitBox


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



##
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:
   why is it 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:
us...@infra.apache.org



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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-23 Thread GitBox


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



##
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:
   shall we put a resolved v1 relation 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:
us...@infra.apache.org



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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-23 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
##
@@ -731,3 +732,11 @@ case class HiveTableRelation(
 s"$nodeName $metadataStr"
   }
 }
+
+/**
+ * A V2 table with V1 fallback support. This is used to fallback to V1 table 
when the V2 one
+ * doesn't implement specific capabilities but V1 already has.
+ */
+trait V2TableWithV1Fallback extends Table {

Review comment:
   shall we define it in the same file as `V1Table`?





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-23 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -1020,16 +1039,35 @@ class Analyzer(
 // 3) If a v1 table is found, create a v1 relation. Otherwise, create a v2 
relation.
 private def lookupRelation(
 identifier: Seq[String],
-options: CaseInsensitiveStringMap): Option[LogicalPlan] = {
+options: CaseInsensitiveStringMap,
+isStreaming: Boolean): Option[LogicalPlan] = {
   expandRelationName(identifier) match {
 case SessionCatalogAndIdentifier(catalog, ident) =>
   lazy val loaded = CatalogV2Util.loadTable(catalog, ident).map {
 case v1Table: V1Table =>
-  v1SessionCatalog.getRelation(v1Table.v1Table, options)
+  if (isStreaming) {
+SubqueryAlias(

Review comment:
   what happens for views? shall we fail here if `v1Table` is a view before 
we have a good story?





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-21 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -895,10 +896,12 @@ class Analyzer(
   object ResolveTables extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = 
ResolveTempViews(plan).resolveOperatorsUp {
   case u: UnresolvedRelation =>
-lookupV2Relation(u.multipartIdentifier, u.options)
-  .map { rel =>
-val ident = rel.identifier.get
-SubqueryAlias(rel.catalog.get.name +: ident.namespace :+ 
ident.name, rel)
+lookupV2Relation(u.multipartIdentifier, u.options, u.isStreaming)
+  .map {
+case rel: DataSourceV2Relation =>
+  val ident = rel.identifier.get
+  SubqueryAlias(rel.catalog.get.name +: ident.namespace :+ 
ident.name, rel)

Review comment:
   shouldn't we add `SubqueryAlias` to streaming relation as well?

##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -1020,16 +1031,28 @@ class Analyzer(
 // 3) If a v1 table is found, create a v1 relation. Otherwise, create a v2 
relation.
 private def lookupRelation(
 identifier: Seq[String],
-options: CaseInsensitiveStringMap): Option[LogicalPlan] = {
+options: CaseInsensitiveStringMap,
+isStreaming: Boolean): Option[LogicalPlan] = {
   expandRelationName(identifier) match {
 case SessionCatalogAndIdentifier(catalog, ident) =>
   lazy val loaded = CatalogV2Util.loadTable(catalog, ident).map {
 case v1Table: V1Table =>
-  v1SessionCatalog.getRelation(v1Table.v1Table, options)
+  if (isStreaming) {
+UnresolvedCatalogRelation(v1Table.v1Table, options, 
isStreaming = true)
+  } else {
+v1SessionCatalog.getRelation(v1Table.v1Table, options)
+  }
 case table =>
-  SubqueryAlias(
-catalog.name +: ident.asMultipartIdentifier,
-DataSourceV2Relation.create(table, Some(catalog), Some(ident), 
options))
+  if (isStreaming) {
+val tableMeta = 
v1SessionCatalog.getTableMetadata(ident.asTableIdentifier)
+StreamingRelationV2(
+  None, table.name, table, options, table.schema.toAttributes,
+  Some(UnresolvedCatalogRelation(tableMeta, isStreaming = 
true)))
+  } else {
+SubqueryAlias(

Review comment:
   ditto

##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
##
@@ -0,0 +1,178 @@
+/*
+ * 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.analysis.TableAlreadyExistsException
+import org.apache.spark.sql.connector.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}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.StreamTest
+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 

[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-21 Thread GitBox


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



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
##
@@ -0,0 +1,178 @@
+/*
+ * 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.analysis.TableAlreadyExistsException
+import org.apache.spark.sql.connector.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}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.StreamTest
+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("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 = 

[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-21 Thread GitBox


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



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
##
@@ -0,0 +1,178 @@
+/*
+ * 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.analysis.TableAlreadyExistsException
+import org.apache.spark.sql.connector.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}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.StreamTest
+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") {

Review comment:
   if we read a batch temp view using this API, shall we fail?





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-21 Thread GitBox


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



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala
##
@@ -0,0 +1,178 @@
+/*
+ * 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.analysis.TableAlreadyExistsException
+import org.apache.spark.sql.connector.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}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.StreamTest
+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("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 = 

[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-21 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -1020,16 +1031,28 @@ class Analyzer(
 // 3) If a v1 table is found, create a v1 relation. Otherwise, create a v2 
relation.
 private def lookupRelation(
 identifier: Seq[String],
-options: CaseInsensitiveStringMap): Option[LogicalPlan] = {
+options: CaseInsensitiveStringMap,
+isStreaming: Boolean): Option[LogicalPlan] = {
   expandRelationName(identifier) match {
 case SessionCatalogAndIdentifier(catalog, ident) =>
   lazy val loaded = CatalogV2Util.loadTable(catalog, ident).map {
 case v1Table: V1Table =>
-  v1SessionCatalog.getRelation(v1Table.v1Table, options)
+  if (isStreaming) {
+UnresolvedCatalogRelation(v1Table.v1Table, options, 
isStreaming = true)
+  } else {
+v1SessionCatalog.getRelation(v1Table.v1Table, options)
+  }
 case table =>
-  SubqueryAlias(
-catalog.name +: ident.asMultipartIdentifier,
-DataSourceV2Relation.create(table, Some(catalog), Some(ident), 
options))
+  if (isStreaming) {
+val tableMeta = 
v1SessionCatalog.getTableMetadata(ident.asTableIdentifier)
+StreamingRelationV2(
+  None, table.name, table, options, table.schema.toAttributes,
+  Some(UnresolvedCatalogRelation(tableMeta, isStreaming = 
true)))
+  } else {
+SubqueryAlias(

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



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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-21 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -895,10 +896,12 @@ class Analyzer(
   object ResolveTables extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = 
ResolveTempViews(plan).resolveOperatorsUp {
   case u: UnresolvedRelation =>
-lookupV2Relation(u.multipartIdentifier, u.options)
-  .map { rel =>
-val ident = rel.identifier.get
-SubqueryAlias(rel.catalog.get.name +: ident.namespace :+ 
ident.name, rel)
+lookupV2Relation(u.multipartIdentifier, u.options, u.isStreaming)
+  .map {
+case rel: DataSourceV2Relation =>
+  val ident = rel.identifier.get
+  SubqueryAlias(rel.catalog.get.name +: ident.namespace :+ 
ident.name, rel)

Review comment:
   shouldn't we add `SubqueryAlias` to streaming relation 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:
us...@infra.apache.org



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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-17 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -846,9 +847,9 @@ class Analyzer(
*/
   object ResolveTempViews extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
-  case u @ UnresolvedRelation(ident, _) =>
+  case u @ UnresolvedRelation(ident, _, _) =>
 lookupTempView(ident).getOrElse(u)

Review comment:
   I think it's less confusing if `DataFrameReader.table` fails on reading 
streaming temp view while `SparkSession.table` works. But it's a breaking 
change and we at least shouldn't do it in this PR.
   
   





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-17 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -846,9 +847,9 @@ class Analyzer(
*/
   object ResolveTempViews extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
-  case u @ UnresolvedRelation(ident, _) =>
+  case u @ UnresolvedRelation(ident, _, _) =>
 lookupTempView(ident).getOrElse(u)

Review comment:
   I'm not sure about the other way around. We have `SparkSession.table`, 
which can read both batch or streaming temp views. We shouldn't break it.
   
   It's a bit weird if `DataFramaReader.table` can read streaming temp view, 
but this is the existing behavior and probably is fine.





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-17 Thread GitBox


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



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
##
@@ -78,6 +78,22 @@ class RateStreamProviderSuite extends StreamTest {
 )
   }
 
+  test("stream table API support") {

Review comment:
   can we move `DataStreamTableAPISuite` to a new file and move this test 
to there 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:
us...@infra.apache.org



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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-17 Thread GitBox


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



##
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, _, _), 
_, _, _, _) =>

Review comment:
   also require `false` 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:
us...@infra.apache.org



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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-17 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
##
@@ -174,7 +174,7 @@ case class CreateViewCommand(
   def verify(child: LogicalPlan) {
 child.collect {
   // Disallow creating permanent views based on temporary views.
-  case UnresolvedRelation(nameParts, _) if 
catalog.isTempView(nameParts) =>
+  case UnresolvedRelation(nameParts, _, _) if 
catalog.isTempView(nameParts) =>

Review comment:
   +1





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-17 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
##
@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, 
ExpressionInfo, Im
 import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, 
ParserInterface}
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
SubqueryAlias, View}
 import org.apache.spark.sql.catalyst.util.StringUtils
-import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier}

Review comment:
   unnecessary change?





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-17 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -846,9 +847,9 @@ class Analyzer(
*/
   object ResolveTempViews extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
-  case u @ UnresolvedRelation(ident, _) =>
+  case u @ UnresolvedRelation(ident, _, _) =>
 lookupTempView(ident).getOrElse(u)

Review comment:
   shall we fail if the temp view is not a streaming plan but the 
`isStreaming` flag is true?





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-16 Thread GitBox


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



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

Review comment:
   nit: should make sure `isStreaming=false` 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:
us...@infra.apache.org



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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-16 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
##
@@ -475,6 +475,16 @@ class SessionCatalog(
 externalCatalog.getTable(db, table)
   }
 
+  /**
+   * Retrieve the metadata of a catalog identifier.
+   */
+  def getTableMetadata(ident: Identifier): CatalogTable = {

Review comment:
   shall we turn `Identifier` into `TableIdentifier` and call the existing 
`getTableMetadata` method?





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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-16 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -942,6 +957,47 @@ class Analyzer(
   }
   }
 
+  /**
+   * Replace [[UnresolvedRelation]] with concrete streaming logical plans.
+   */
+  object ResolveStreamingRelation extends Rule[LogicalPlan] {

Review comment:
   shall we merge it into `ResolveRelations` to avoid duplicated 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:
us...@infra.apache.org



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



[GitHub] [spark] cloud-fan commented on a change in pull request #29756: [SPARK-32885][SS] Add DataStreamReader.table API

2020-09-16 Thread GitBox


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



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##
@@ -923,6 +925,19 @@ class Analyzer(
 
   case u: UnresolvedV2Relation =>
 CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u)
+
+  case u @ UnresolvedRelation(_, extraOptions, true) =>
+val r = expandRelationName(u.multipartIdentifier) match {
+  case NonSessionCatalogAndIdentifier(catalog, ident) =>
+CatalogV2Util.loadTable(catalog, ident) match {
+  case Some(table) =>
+Some(StreamingRelationV2(

Review comment:
   shall we just add a `isStreaming` flag in `lookupV2Relation`, to unify 
the code a bit more?





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