From ac8b0b92a66f4d905507d0fdfc752a41b1cf9f51 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 19 Dec 2019 17:43:03 -0800 Subject: [PATCH 01/23] modify case class signature and all dependencies --- .../sql/kafka010/KafkaRelationSuite.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../sql/connector/catalog/CatalogV2Util.scala | 2 +- .../datasources/v2/DataSourceV2Relation.scala | 18 ++++-- .../apache/spark/sql/DataFrameReader.scala | 5 +- .../apache/spark/sql/DataFrameWriter.scala | 8 ++- .../apache/spark/sql/DataFrameWriterV2.scala | 10 +++- .../spark/sql/execution/CacheManager.scala | 2 +- .../datasources/FallBackFileSourceV2.scala | 3 +- .../datasources/v2/DataSourceV2Strategy.scala | 4 +- .../connector/TableCapabilityCheckSuite.scala | 56 ++++++++++++++++--- .../ParquetPartitionDiscoverySuite.scala | 2 +- .../sql/streaming/FileStreamSinkSuite.scala | 2 +- 13 files changed, 89 insertions(+), 29 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 063e2e2bc8b7..efb4eb057d8c 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -624,7 +624,7 @@ class KafkaRelationSuiteV2 extends KafkaRelationSuiteBase { val topic = newTopic() val df = createDF(topic) assert(df.logicalPlan.collect { - case DataSourceV2Relation(_, _, _) => true + case DataSourceV2Relation(_, _, _, _, _) => true }.nonEmpty) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 2438ef921822..cde73a3d86d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -784,7 +784,7 @@ class Analyzer( identifier match { case NonSessionCatalogAndIdentifier(catalog, ident) => CatalogV2Util.loadTable(catalog, ident) match { - case Some(table) => Some(DataSourceV2Relation.create(table)) + case Some(table) => Some(DataSourceV2Relation.create(table, Some(catalog), Seq(ident))) case None => None } case _ => None @@ -886,7 +886,7 @@ class Analyzer( Some(relation) } case Some(table) => - Some(DataSourceV2Relation.create(table)) + Some(DataSourceV2Relation.create(table, Some(catalog), Seq(ident))) case None => None } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 2f4914dd7db3..5865c702c080 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -256,7 +256,7 @@ private[sql] object CatalogV2Util { } def loadRelation(catalog: CatalogPlugin, ident: Identifier): Option[NamedRelation] = { - loadTable(catalog, ident).map(DataSourceV2Relation.create) + loadTable(catalog, ident).map(DataSourceV2Relation.create(_, Some(catalog), Seq(ident))) } def isSessionCatalog(catalog: CatalogPlugin): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 87d3419e8115..797369d3bc1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelat import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.connector.catalog.{Table, TableCapability} +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, Table, TableCapability} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, Statistics => V2Statistics, SupportsReportStatistics} import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.connector.write.WriteBuilder @@ -38,6 +38,8 @@ import org.apache.spark.util.Utils case class DataSourceV2Relation( table: Table, output: Seq[AttributeReference], + catalog: Option[CatalogPlugin], + identifiers: Seq[Identifier], options: CaseInsensitiveStringMap) extends LeafNode with MultiInstanceRelation with NamedRelation { @@ -137,12 +139,20 @@ case class StreamingDataSourceV2Relation( } object DataSourceV2Relation { - def create(table: Table, options: CaseInsensitiveStringMap): DataSourceV2Relation = { + def create( + table: Table, + catalog: Option[CatalogPlugin], + identifiers: Seq[Identifier], + options: CaseInsensitiveStringMap): DataSourceV2Relation = { val output = table.schema().toAttributes - DataSourceV2Relation(table, output, options) + DataSourceV2Relation(table, output, catalog, identifiers, options) } - def create(table: Table): DataSourceV2Relation = create(table, CaseInsensitiveStringMap.empty) + def create( + table: Table, + catalog: Option[CatalogPlugin], + identifiers: Seq[Identifier]): DataSourceV2Relation = + create(table, catalog, identifiers, CaseInsensitiveStringMap.empty) /** * This is used to transform data source v2 statistics to logical.Statistics. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 8570e4640fee..f5f660b3db9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -222,7 +222,10 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ table match { case _: SupportsRead if table.supports(BATCH_READ) => - Dataset.ofRows(sparkSession, DataSourceV2Relation.create(table, dsOptions)) + // TODO: Pass the PathIdentifiers as the list to V2Relation once that's implemented. + Dataset.ofRows( + sparkSession, + DataSourceV2Relation.create(table, None, Seq.empty, dsOptions)) case _ => loadV1Source(paths: _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 2b124ae260ca..c4ce48a1a3ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -264,7 +264,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { throw new AnalysisException("Cannot write data to TableProvider implementation " + "if partition columns are specified.") } - lazy val relation = DataSourceV2Relation.create(table, dsOptions) + lazy val relation = DataSourceV2Relation.create(table, None, Seq.empty, dsOptions) mode match { case SaveMode.Append => runCommand(df.sparkSession, "save") { @@ -379,7 +379,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case _: V1Table => return insertInto(TableIdentifier(ident.name(), ident.namespace().headOption)) case t => - DataSourceV2Relation.create(t) + DataSourceV2Relation.create(t, Some(catalog), Seq(ident)) } val command = mode match { @@ -526,7 +526,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { return saveAsTable(TableIdentifier(ident.name(), ident.namespace().headOption)) case (SaveMode.Append, Some(table)) => - AppendData.byName(DataSourceV2Relation.create(table), df.logicalPlan, extraOptions.toMap) + AppendData.byName( + DataSourceV2Relation.create(table, Some(catalog), Seq(ident)), + df.logicalPlan, extraOptions.toMap) case (SaveMode.Overwrite, _) => ReplaceTableAsSelect( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index f0758809bd74..223ef987b8e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -158,7 +158,9 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) def append(): Unit = { val append = loadTable(catalog, identifier) match { case Some(t) => - AppendData.byName(DataSourceV2Relation.create(t), logicalPlan, options.toMap) + AppendData.byName( + DataSourceV2Relation.create(t, Some(catalog), Seq(identifier)), + logicalPlan, options.toMap) case _ => throw new NoSuchTableException(identifier) } @@ -181,7 +183,8 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) val overwrite = loadTable(catalog, identifier) match { case Some(t) => OverwriteByExpression.byName( - DataSourceV2Relation.create(t), logicalPlan, condition.expr, options.toMap) + DataSourceV2Relation.create(t, Some(catalog), Seq(identifier)), + logicalPlan, condition.expr, options.toMap) case _ => throw new NoSuchTableException(identifier) } @@ -207,7 +210,8 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) val dynamicOverwrite = loadTable(catalog, identifier) match { case Some(t) => OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(t), logicalPlan, options.toMap) + DataSourceV2Relation.create(t, Some(catalog), Seq(identifier)), + logicalPlan, options.toMap) case _ => throw new NoSuchTableException(identifier) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 10dc74dd8a8f..3b824397abf8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -261,7 +261,7 @@ class CacheManager extends Logging { case _ => false } - case DataSourceV2Relation(fileTable: FileTable, _, _) => + case DataSourceV2Relation(fileTable: FileTable, _, _, _, _) => refreshFileIndexIfNecessary(fileTable.fileIndex, fs, qualifiedPath) case _ => false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala index 5f6c3e8f7eee..28a63c26604e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala @@ -33,7 +33,8 @@ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, File */ class FallBackFileSourceV2(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @ InsertIntoStatement(d @ DataSourceV2Relation(table: FileTable, _, _), _, _, _, _) => + case i @ + InsertIntoStatement(d @ DataSourceV2Relation(table: FileTable, _, _, _, _), _, _, _, _) => val v1FileFormat = table.fallbackFileFormat.newInstance() val relation = HadoopFsRelation( table.fileIndex, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 44de8f275fea..64ebeef7120e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -195,7 +195,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case desc @ DescribeNamespace(catalog, namespace, extended) => DescribeNamespaceExec(desc.output, catalog, namespace, extended) :: Nil - case desc @ DescribeTable(DataSourceV2Relation(table, _, _), isExtended) => + case desc @ DescribeTable(DataSourceV2Relation(table, _, _, _, _), isExtended) => DescribeTableExec(desc.output, table, isExtended) :: Nil case DropTable(catalog, ident, ifExists) => @@ -228,7 +228,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case r: ShowCurrentNamespace => ShowCurrentNamespaceExec(r.output, r.catalogManager) :: Nil - case r @ ShowTableProperties(DataSourceV2Relation(table, _, _), propertyKey) => + case r @ ShowTableProperties(DataSourceV2Relation(table, _, _, _, _), propertyKey) => ShowTablePropertiesExec(r.output, table, propertyKey) :: Nil case _ => Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index ce6d56cf84df..3cb5b7eba075 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -55,6 +55,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { val e = intercept[AnalysisException] { TableCapabilityCheck.apply(DataSourceV2Relation.create( CapabilityTable(), + None, + Seq.empty, CaseInsensitiveStringMap.empty)) } assert(e.message.contains("does not support batch scan")) @@ -88,7 +90,12 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("AppendData: check missing capabilities") { val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(), CaseInsensitiveStringMap.empty), TestRelation) + DataSourceV2Relation.create( + CapabilityTable(), + None, + Seq.empty, + CaseInsensitiveStringMap.empty), + TestRelation) val exc = intercept[AnalysisException]{ TableCapabilityCheck.apply(plan) @@ -100,7 +107,11 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("AppendData: check correct capabilities") { Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(write), CaseInsensitiveStringMap.empty), + DataSourceV2Relation.create( + CapabilityTable(write), + None, + Seq.empty, + CaseInsensitiveStringMap.empty), TestRelation) TableCapabilityCheck.apply(plan) @@ -115,7 +126,12 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + DataSourceV2Relation.create( + table, + None, + Seq.empty, + CaseInsensitiveStringMap.empty), + TestRelation, Literal(true)) val exc = intercept[AnalysisException]{ @@ -133,7 +149,11 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(V1_BATCH_WRITE, OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + DataSourceV2Relation.create( + table, + None, + Seq.empty, + CaseInsensitiveStringMap.empty), TestRelation, Literal(true)) TableCapabilityCheck.apply(plan) @@ -147,7 +167,12 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + DataSourceV2Relation.create( + table, + None, + Seq.empty, + CaseInsensitiveStringMap.empty), + TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) val exc = intercept[AnalysisException]{ @@ -162,7 +187,12 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val table = CapabilityTable(write, OVERWRITE_BY_FILTER) val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + DataSourceV2Relation.create( + table, + None, + Seq.empty, + CaseInsensitiveStringMap.empty), + TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) TableCapabilityCheck.apply(plan) @@ -175,7 +205,12 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_DYNAMIC)).foreach { table => val plan = OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation) + DataSourceV2Relation.create( + table, + None, + Seq.empty, + CaseInsensitiveStringMap.empty), + TestRelation) val exc = intercept[AnalysisException] { TableCapabilityCheck.apply(plan) @@ -188,7 +223,12 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("OverwritePartitionsDynamic: check correct capabilities") { val table = CapabilityTable(BATCH_WRITE, OVERWRITE_DYNAMIC) val plan = OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation) + DataSourceV2Relation.create( + table, + None, + Seq.empty, + CaseInsensitiveStringMap.empty), + TestRelation) TableCapabilityCheck.apply(plan) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 6497a1ceb5c0..e63929470ce5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -1281,7 +1281,7 @@ class ParquetV2PartitionDiscoverySuite extends ParquetPartitionDiscoverySuite { (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case DataSourceV2Relation(fileTable: FileTable, _, _) => + case DataSourceV2Relation(fileTable: FileTable, _, _, _, _) => assert(fileTable.fileIndex.partitionSpec() === PartitionSpec.emptySpec) }.getOrElse { fail(s"Expecting a matching DataSourceV2Relation, but got:\n$queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 9bce7f3568e8..877965100f01 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -657,7 +657,7 @@ class FileStreamSinkV2Suite extends FileStreamSinkSuite { // Verify that MetadataLogFileIndex is being used and the correct partitioning schema has // been inferred val table = df.queryExecution.analyzed.collect { - case DataSourceV2Relation(table: FileTable, _, _) => table + case DataSourceV2Relation(table: FileTable, _, _, _, _) => table } assert(table.size === 1) assert(table.head.fileIndex.isInstanceOf[MetadataLogFileIndex]) From d4c9ba51f6988d9e03fe3d5f2ef7de707fe5a305 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 19 Dec 2019 21:28:50 -0800 Subject: [PATCH 02/23] use catalog identifier instead of catalogPlugin --- .../sql/catalyst/analysis/Analyzer.scala | 17 +++++++---- .../sql/connector/catalog/CatalogV2Util.scala | 7 +++-- .../datasources/v2/DataSourceV2Relation.scala | 10 +++---- .../apache/spark/sql/DataFrameWriter.scala | 30 ++++++++++++------- .../apache/spark/sql/DataFrameWriterV2.scala | 19 +++++++----- 5 files changed, 52 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index cde73a3d86d9..6ce1fc0bbf50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -759,22 +759,23 @@ class Analyzer( .getOrElse(i) case desc @ DescribeTable(u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.tableName) + CatalogV2Util.loadRelation(u.catalog, u.originalNameParts.headOption, u.tableName) .map(rel => desc.copy(table = rel)) .getOrElse(desc) case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.tableName) + CatalogV2Util.loadRelation(u.catalog, u.originalNameParts.headOption, u.tableName) .map(rel => alter.copy(table = rel)) .getOrElse(alter) case show @ ShowTableProperties(u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.tableName) + CatalogV2Util.loadRelation(u.catalog, u.originalNameParts.headOption, u.tableName) .map(rel => show.copy(table = rel)) .getOrElse(show) case u: UnresolvedV2Relation => - CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u) + CatalogV2Util.loadRelation(u.catalog, u.originalNameParts.headOption, u.tableName) + .getOrElse(u) } /** @@ -784,7 +785,8 @@ class Analyzer( identifier match { case NonSessionCatalogAndIdentifier(catalog, ident) => CatalogV2Util.loadTable(catalog, ident) match { - case Some(table) => Some(DataSourceV2Relation.create(table, Some(catalog), Seq(ident))) + case Some(table) => + Some(DataSourceV2Relation.create(table, identifier.headOption, Seq(ident))) case None => None } case _ => None @@ -886,7 +888,10 @@ class Analyzer( Some(relation) } case Some(table) => - Some(DataSourceV2Relation.create(table, Some(catalog), Seq(ident))) + Some(DataSourceV2Relation.create( + table, + Some(CatalogManager.SESSION_CATALOG_NAME), + Seq(newIdent))) case None => None } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 5865c702c080..70509aba5632 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -255,8 +255,11 @@ private[sql] object CatalogV2Util { case _: NoSuchNamespaceException => None } - def loadRelation(catalog: CatalogPlugin, ident: Identifier): Option[NamedRelation] = { - loadTable(catalog, ident).map(DataSourceV2Relation.create(_, Some(catalog), Seq(ident))) + def loadRelation( + catalog: CatalogPlugin, + catalogIdentifier: Option[String], + ident: Identifier): Option[NamedRelation] = { + loadTable(catalog, ident).map(DataSourceV2Relation.create(_, catalogIdentifier, Seq(ident))) } def isSessionCatalog(catalog: CatalogPlugin): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 797369d3bc1e..f64965f68525 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -38,7 +38,7 @@ import org.apache.spark.util.Utils case class DataSourceV2Relation( table: Table, output: Seq[AttributeReference], - catalog: Option[CatalogPlugin], + catalogIdentifier: Option[String], identifiers: Seq[Identifier], options: CaseInsensitiveStringMap) extends LeafNode with MultiInstanceRelation with NamedRelation { @@ -141,18 +141,18 @@ case class StreamingDataSourceV2Relation( object DataSourceV2Relation { def create( table: Table, - catalog: Option[CatalogPlugin], + catalogIdentifier: Option[String], identifiers: Seq[Identifier], options: CaseInsensitiveStringMap): DataSourceV2Relation = { val output = table.schema().toAttributes - DataSourceV2Relation(table, output, catalog, identifiers, options) + DataSourceV2Relation(table, output, catalogIdentifier, identifiers, options) } def create( table: Table, - catalog: Option[CatalogPlugin], + catalogIdentifier: Option[String], identifiers: Seq[Identifier]): DataSourceV2Relation = - create(table, catalog, identifiers, CaseInsensitiveStringMap.empty) + create(table, catalogIdentifier, identifiers, CaseInsensitiveStringMap.empty) /** * This is used to transform data source v2 statistics to logical.Statistics. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index c4ce48a1a3ed..f2241a3b4826 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, InsertIntoStatement, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, SupportsWrite, TableCatalog, TableProvider, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, SupportsWrite, TableCatalog, TableProvider, V1Table} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, LiteralValue, Transform} import org.apache.spark.sql.execution.SQLExecution @@ -356,13 +356,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined - session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { + val multipartIdentifier = session.sessionState.sqlParser.parseMultipartIdentifier(tableName) + multipartIdentifier match { case NonSessionCatalogAndIdentifier(catalog, ident) => - insertInto(catalog, ident) + insertInto(catalog, multipartIdentifier.headOption, ident) case SessionCatalogAndIdentifier(catalog, ident) if canUseV2 && ident.namespace().length <= 1 => - insertInto(catalog, ident) + insertInto(catalog, Some(CatalogManager.SESSION_CATALOG_NAME), ident) case AsTableIdentifier(tableIdentifier) => insertInto(tableIdentifier) @@ -372,14 +373,17 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } - private def insertInto(catalog: CatalogPlugin, ident: Identifier): Unit = { + private def insertInto( + catalog: CatalogPlugin, + catalogIdentifier: Option[String], + ident: Identifier): Unit = { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val table = catalog.asTableCatalog.loadTable(ident) match { case _: V1Table => return insertInto(TableIdentifier(ident.name(), ident.namespace().headOption)) case t => - DataSourceV2Relation.create(t, Some(catalog), Seq(ident)) + DataSourceV2Relation.create(t, catalogIdentifier, Seq(ident)) } val command = mode match { @@ -485,13 +489,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined - session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { + val multipartIdentifier = session.sessionState.sqlParser.parseMultipartIdentifier(tableName) + multipartIdentifier match { case NonSessionCatalogAndIdentifier(catalog, ident) => - saveAsTable(catalog.asTableCatalog, ident) + saveAsTable(catalog.asTableCatalog, multipartIdentifier.headOption, ident) case SessionCatalogAndIdentifier(catalog, ident) if canUseV2 && ident.namespace().length <= 1 => - saveAsTable(catalog.asTableCatalog, ident) + saveAsTable(catalog.asTableCatalog, Some(CatalogManager.SESSION_CATALOG_NAME), ident) case AsTableIdentifier(tableIdentifier) => saveAsTable(tableIdentifier) @@ -503,7 +508,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } - private def saveAsTable(catalog: TableCatalog, ident: Identifier): Unit = { + private def saveAsTable( + catalog: TableCatalog, + catalogIdentifier: Option[String], + ident: Identifier): Unit = { val partitioning = partitioningColumns.map { colNames => colNames.map(name => IdentityTransform(FieldReference(name))) }.getOrElse(Seq.empty[Transform]) @@ -527,7 +535,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case (SaveMode.Append, Some(table)) => AppendData.byName( - DataSourceV2Relation.create(table, Some(catalog), Seq(ident)), + DataSourceV2Relation.create(table, catalogIdentifier, Seq(ident)), df.logicalPlan, extraOptions.toMap) case (SaveMode.Overwrite, _) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index 223ef987b8e3..7d1689843826 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -24,7 +24,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.expressions.{Attribute, Bucket, Days, Hours, Literal, Months, Years} import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} -import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.connector.catalog.{CatalogManager, TableCatalog} import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference, Transform} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation @@ -51,9 +51,14 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table) - private val (catalog, identifier) = { - val CatalogAndIdentifier(catalog, identifier) = tableName - (catalog.asTableCatalog, identifier) + private val (catalog, catalogIdentifier, identifier) = { + import df.sparkSession.sessionState.analyzer.{NonSessionCatalogAndIdentifier, SessionCatalogAndIdentifier} + tableName match { + case NonSessionCatalogAndIdentifier(catalog, identifier) => + (catalog.asTableCatalog, tableName.headOption, identifier) + case SessionCatalogAndIdentifier(catalog, identifier) => + (catalog.asTableCatalog, Some(CatalogManager.SESSION_CATALOG_NAME), identifier) + } } private val logicalPlan = df.queryExecution.logical @@ -159,7 +164,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) val append = loadTable(catalog, identifier) match { case Some(t) => AppendData.byName( - DataSourceV2Relation.create(t, Some(catalog), Seq(identifier)), + DataSourceV2Relation.create(t, catalogIdentifier, Seq(identifier)), logicalPlan, options.toMap) case _ => throw new NoSuchTableException(identifier) @@ -183,7 +188,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) val overwrite = loadTable(catalog, identifier) match { case Some(t) => OverwriteByExpression.byName( - DataSourceV2Relation.create(t, Some(catalog), Seq(identifier)), + DataSourceV2Relation.create(t, catalogIdentifier, Seq(identifier)), logicalPlan, condition.expr, options.toMap) case _ => throw new NoSuchTableException(identifier) @@ -210,7 +215,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) val dynamicOverwrite = loadTable(catalog, identifier) match { case Some(t) => OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(t, Some(catalog), Seq(identifier)), + DataSourceV2Relation.create(t, catalogIdentifier, Seq(identifier)), logicalPlan, options.toMap) case _ => throw new NoSuchTableException(identifier) From cf2cef25ae04c2c316dfba26090dbf117e566d94 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Mon, 23 Dec 2019 20:50:08 -0800 Subject: [PATCH 03/23] refactor --- .../sql/catalyst/analysis/Analyzer.scala | 27 ++++++++------ .../connector/catalog/CatalogManager.scala | 20 +++++++++-- .../apache/spark/sql/DataFrameWriter.scala | 35 +++++++++---------- 3 files changed, 52 insertions(+), 30 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6ce1fc0bbf50..3a8d50436be2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -759,22 +759,26 @@ class Analyzer( .getOrElse(i) case desc @ DescribeTable(u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.originalNameParts.headOption, u.tableName) - .map(rel => desc.copy(table = rel)) - .getOrElse(desc) + CatalogV2Util + .loadRelation(u.catalog, catalogManager.catalogIdentifier(u.catalog), u.tableName) + .map(rel => desc.copy(table = rel)) + .getOrElse(desc) case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.originalNameParts.headOption, u.tableName) - .map(rel => alter.copy(table = rel)) - .getOrElse(alter) + CatalogV2Util + .loadRelation(u.catalog, catalogManager.catalogIdentifier(u.catalog), u.tableName) + .map(rel => alter.copy(table = rel)) + .getOrElse(alter) case show @ ShowTableProperties(u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.originalNameParts.headOption, u.tableName) + CatalogV2Util + .loadRelation(u.catalog, catalogManager.catalogIdentifier(u.catalog), u.tableName) .map(rel => show.copy(table = rel)) .getOrElse(show) case u: UnresolvedV2Relation => - CatalogV2Util.loadRelation(u.catalog, u.originalNameParts.headOption, u.tableName) + CatalogV2Util + .loadRelation(u.catalog, catalogManager.catalogIdentifier(u.catalog), u.tableName) .getOrElse(u) } @@ -786,7 +790,10 @@ class Analyzer( case NonSessionCatalogAndIdentifier(catalog, ident) => CatalogV2Util.loadTable(catalog, ident) match { case Some(table) => - Some(DataSourceV2Relation.create(table, identifier.headOption, Seq(ident))) + Some(DataSourceV2Relation.create( + table, + catalogManager.catalogIdentifier(catalog), + Seq(ident))) case None => None } case _ => None @@ -890,7 +897,7 @@ class Analyzer( case Some(table) => Some(DataSourceV2Relation.create( table, - Some(CatalogManager.SESSION_CATALOG_NAME), + catalogManager.catalogIdentifier(catalog), Seq(newIdent))) case None => None } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index 135c180ef400..8d3e2a6ba656 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -44,15 +44,27 @@ class CatalogManager( import CatalogManager.SESSION_CATALOG_NAME private val catalogs = mutable.HashMap.empty[String, CatalogPlugin] + // Map from catalog back to it's original name for easy name loop up, we don't use the + // CatalogPlugin's name as it might be different from the catalog name depending on + // implementation. + private val catalogIdentifiers = mutable.HashMap.empty[CatalogPlugin, String] def catalog(name: String): CatalogPlugin = synchronized { if (name.equalsIgnoreCase(SESSION_CATALOG_NAME)) { v2SessionCatalog } else { - catalogs.getOrElseUpdate(name, Catalogs.load(name, conf)) + catalogs.getOrElseUpdate(name, { + val catalog = Catalogs.load(name, conf) + catalogIdentifiers(catalog) = name + catalog + }) } } + def catalogIdentifier(catalog: CatalogPlugin): Option[String] = synchronized { + catalogIdentifiers.get(catalog) + } + private def loadV2SessionCatalog(): CatalogPlugin = { Catalogs.load(SESSION_CATALOG_NAME, conf) match { case extension: CatalogExtension => @@ -74,7 +86,11 @@ class CatalogManager( private[sql] def v2SessionCatalog: CatalogPlugin = { conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).map { customV2SessionCatalog => try { - catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) + catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, { + val catalog = loadV2SessionCatalog() + catalogIdentifiers(catalog) = SESSION_CATALOG_NAME + catalog + }) } catch { case NonFatal(_) => logError( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index f2241a3b4826..b0c82adf9028 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -264,6 +264,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { throw new AnalysisException("Cannot write data to TableProvider implementation " + "if partition columns are specified.") } + // TODO: pass catalog and table identifiers to V2Relation lazy val relation = DataSourceV2Relation.create(table, None, Seq.empty, dsOptions) mode match { case SaveMode.Append => @@ -356,14 +357,13 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined - val multipartIdentifier = session.sessionState.sqlParser.parseMultipartIdentifier(tableName) - multipartIdentifier match { + session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { case NonSessionCatalogAndIdentifier(catalog, ident) => - insertInto(catalog, multipartIdentifier.headOption, ident) + insertInto(catalog, ident) case SessionCatalogAndIdentifier(catalog, ident) if canUseV2 && ident.namespace().length <= 1 => - insertInto(catalog, Some(CatalogManager.SESSION_CATALOG_NAME), ident) + insertInto(catalog, ident) case AsTableIdentifier(tableIdentifier) => insertInto(tableIdentifier) @@ -373,17 +373,15 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } - private def insertInto( - catalog: CatalogPlugin, - catalogIdentifier: Option[String], - ident: Identifier): Unit = { + private def insertInto(catalog: CatalogPlugin, ident: Identifier): Unit = { + import df.sparkSession.sessionState.analyzer.catalogManager import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val table = catalog.asTableCatalog.loadTable(ident) match { case _: V1Table => return insertInto(TableIdentifier(ident.name(), ident.namespace().headOption)) case t => - DataSourceV2Relation.create(t, catalogIdentifier, Seq(ident)) + DataSourceV2Relation.create(t, catalogManager.catalogIdentifier(catalog), Seq(ident)) } val command = mode match { @@ -489,14 +487,13 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined - val multipartIdentifier = session.sessionState.sqlParser.parseMultipartIdentifier(tableName) - multipartIdentifier match { + session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { case NonSessionCatalogAndIdentifier(catalog, ident) => - saveAsTable(catalog.asTableCatalog, multipartIdentifier.headOption, ident) + saveAsTable(catalog.asTableCatalog, ident) case SessionCatalogAndIdentifier(catalog, ident) if canUseV2 && ident.namespace().length <= 1 => - saveAsTable(catalog.asTableCatalog, Some(CatalogManager.SESSION_CATALOG_NAME), ident) + saveAsTable(catalog.asTableCatalog, ident) case AsTableIdentifier(tableIdentifier) => saveAsTable(tableIdentifier) @@ -508,10 +505,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } - private def saveAsTable( - catalog: TableCatalog, - catalogIdentifier: Option[String], - ident: Identifier): Unit = { + private def saveAsTable(catalog: TableCatalog, ident: Identifier): Unit = { + import df.sparkSession.sessionState.analyzer.catalogManager + val partitioning = partitioningColumns.map { colNames => colNames.map(name => IdentityTransform(FieldReference(name))) }.getOrElse(Seq.empty[Transform]) @@ -535,7 +531,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case (SaveMode.Append, Some(table)) => AppendData.byName( - DataSourceV2Relation.create(table, catalogIdentifier, Seq(ident)), + DataSourceV2Relation.create( + table, + catalogManager.catalogIdentifier(catalog), + Seq(ident)), df.logicalPlan, extraOptions.toMap) case (SaveMode.Overwrite, _) => From 374def942f1bbaac874e00f8264581f0dc522aba Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Mon, 23 Dec 2019 20:55:09 -0800 Subject: [PATCH 04/23] refactor --- .../sql/execution/datasources/v2/DataSourceV2Strategy.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 64ebeef7120e..961bb5251f51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -195,8 +195,8 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case desc @ DescribeNamespace(catalog, namespace, extended) => DescribeNamespaceExec(desc.output, catalog, namespace, extended) :: Nil - case desc @ DescribeTable(DataSourceV2Relation(table, _, _, _, _), isExtended) => - DescribeTableExec(desc.output, table, isExtended) :: Nil + case desc @ DescribeTable(r: DataSourceV2Relation, isExtended) => + DescribeTableExec(desc.output, r.table, isExtended) :: Nil case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil From 65138241b7574148295464d093cca443a4a2c4f1 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Tue, 7 Jan 2020 16:14:51 -0800 Subject: [PATCH 05/23] nits --- .../org/apache/spark/sql/kafka010/KafkaRelationSuite.scala | 2 +- .../apache/spark/sql/connector/catalog/CatalogManager.scala | 2 +- .../sql/execution/datasources/v2/DataSourceV2Relation.scala | 4 ++++ .../sql/execution/datasources/v2/DataSourceV2Strategy.scala | 4 ++-- 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index efb4eb057d8c..2c022c11950b 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -624,7 +624,7 @@ class KafkaRelationSuiteV2 extends KafkaRelationSuiteBase { val topic = newTopic() val df = createDF(topic) assert(df.logicalPlan.collect { - case DataSourceV2Relation(_, _, _, _, _) => true + case _: DataSourceV2Relation => true }.nonEmpty) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index 7060f88e7b31..b00fa80760ef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -44,7 +44,7 @@ class CatalogManager( import CatalogManager.SESSION_CATALOG_NAME private val catalogs = mutable.HashMap.empty[String, CatalogPlugin] - // Map from catalog back to it's original name for easy name loop up, we don't use the + // Map from catalog back to it's original name for easy name look up, we don't use the // CatalogPlugin's name as it might be different from the catalog name depending on // implementation. private val catalogIdentifiers = mutable.HashMap.empty[CatalogPlugin, String] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index f64965f68525..fd984359465f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -32,6 +32,10 @@ import org.apache.spark.util.Utils * A logical plan representing a data source v2 table. * * @param table The table that this relation represents. + * @param output the output attributes of this relation + * @param catalogIdentifier the string identifier for the catalog + * @param identifiers the identifiers for the v2 relation. For multipath dataframe, there could be + * more than one identifier * @param options The options for this table operation. It's used to create fresh [[ScanBuilder]] * and [[WriteBuilder]]. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 03818d9879da..53ceff0f2716 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -240,8 +240,8 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case r: ShowCurrentNamespace => ShowCurrentNamespaceExec(r.output, r.catalogManager) :: Nil - case r @ ShowTableProperties(DataSourceV2Relation(table, _, _, _, _), propertyKey) => - ShowTablePropertiesExec(r.output, table, propertyKey) :: Nil + case r @ ShowTableProperties(v2: DataSourceV2Relation, propertyKey) => + ShowTablePropertiesExec(r.output, v2.table, propertyKey) :: Nil case _ => Nil } From 9b8d4de2d810cb2c539b3c7d866fbf98a64e7f01 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 9 Jan 2020 17:24:29 -0800 Subject: [PATCH 06/23] nits --- .../apache/spark/sql/DataFrameWriterV2.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index 7d1689843826..52357fb1abd1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -51,7 +51,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table) - private val (catalog, catalogIdentifier, identifier) = { + private val (catalog, catalogIdentifier, tableIdentifier) = { import df.sparkSession.sessionState.analyzer.{NonSessionCatalogAndIdentifier, SessionCatalogAndIdentifier} tableName match { case NonSessionCatalogAndIdentifier(catalog, identifier) => @@ -131,7 +131,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) runCommand("create") { CreateTableAsSelect( catalog, - identifier, + tableIdentifier, partitioning.getOrElse(Seq.empty), logicalPlan, properties = provider.map(p => properties + (TableCatalog.PROP_PROVIDER -> p)) @@ -161,13 +161,13 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) */ @throws(classOf[NoSuchTableException]) def append(): Unit = { - val append = loadTable(catalog, identifier) match { + val append = loadTable(catalog, tableIdentifier) match { case Some(t) => AppendData.byName( - DataSourceV2Relation.create(t, catalogIdentifier, Seq(identifier)), + DataSourceV2Relation.create(t, catalogIdentifier, Seq(tableIdentifier)), logicalPlan, options.toMap) case _ => - throw new NoSuchTableException(identifier) + throw new NoSuchTableException(tableIdentifier) } runCommand("append")(append) @@ -185,13 +185,13 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) */ @throws(classOf[NoSuchTableException]) def overwrite(condition: Column): Unit = { - val overwrite = loadTable(catalog, identifier) match { + val overwrite = loadTable(catalog, tableIdentifier) match { case Some(t) => OverwriteByExpression.byName( - DataSourceV2Relation.create(t, catalogIdentifier, Seq(identifier)), + DataSourceV2Relation.create(t, catalogIdentifier, Seq(tableIdentifier)), logicalPlan, condition.expr, options.toMap) case _ => - throw new NoSuchTableException(identifier) + throw new NoSuchTableException(tableIdentifier) } runCommand("overwrite")(overwrite) @@ -212,13 +212,13 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) */ @throws(classOf[NoSuchTableException]) def overwritePartitions(): Unit = { - val dynamicOverwrite = loadTable(catalog, identifier) match { + val dynamicOverwrite = loadTable(catalog, tableIdentifier) match { case Some(t) => OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(t, catalogIdentifier, Seq(identifier)), + DataSourceV2Relation.create(t, catalogIdentifier, Seq(tableIdentifier)), logicalPlan, options.toMap) case _ => - throw new NoSuchTableException(identifier) + throw new NoSuchTableException(tableIdentifier) } runCommand("overwritePartitions")(dynamicOverwrite) @@ -238,7 +238,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) runCommand("replace") { ReplaceTableAsSelect( catalog, - identifier, + tableIdentifier, partitioning.getOrElse(Seq.empty), logicalPlan, properties = provider.map(p => properties + ("provider" -> p)).getOrElse(properties).toMap, From 4625a575379eed364334a4d1f6323bb568727ed3 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 9 Jan 2020 17:39:36 -0800 Subject: [PATCH 07/23] port dataframe reader --- .../org/apache/spark/sql/DataFrameReader.scala | 13 +++++++------ .../org/apache/spark/sql/DataFrameWriter.scala | 11 +++-------- 2 files changed, 10 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index d59ef0cbe276..66872b7e703e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -195,6 +195,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } DataSource.lookupDataSourceV2(source, sparkSession.sessionState.conf).map { provider => + val catalogManager = sparkSession.sessionState.catalogManager val sessionOptions = DataSourceV2Utils.extractSessionConfigs( source = provider, conf = sparkSession.sessionState.conf) val pathsOption = if (paths.isEmpty) { @@ -206,7 +207,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val finalOptions = sessionOptions ++ extraOptions.toMap ++ pathsOption val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava) - val table = provider match { + val (table, catalogOpt, ident) = provider match { case _: SupportsCatalogOptions if userSpecifiedSchema.nonEmpty => throw new IllegalArgumentException( s"$source does not support user specified schema. Please don't specify the schema.") @@ -214,13 +215,13 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val ident = hasCatalog.extractIdentifier(dsOptions) val catalog = CatalogV2Util.getTableProviderCatalog( hasCatalog, - sparkSession.sessionState.catalogManager, + catalogManager, dsOptions) - catalog.loadTable(ident) + (catalog.loadTable(ident), catalogManager.catalogIdentifier(catalog), Seq(ident)) case _ => userSpecifiedSchema match { - case Some(schema) => provider.getTable(dsOptions, schema) - case _ => provider.getTable(dsOptions) + case Some(schema) => (provider.getTable(dsOptions, schema), None, Seq.empty) + case _ => (provider.getTable(dsOptions), None, Seq.empty) } } import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ @@ -229,7 +230,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { // TODO: Pass the PathIdentifiers as the list to V2Relation once that's implemented. Dataset.ofRows( sparkSession, - DataSourceV2Relation.create(table, None, Seq.empty, dsOptions)) + DataSourceV2Relation.create(table, catalogOpt, ident, dsOptions)) case _ => loadV1Source(paths: _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 6669b2ddc8e2..a36a2aa0edca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -258,7 +258,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val dsOptions = new CaseInsensitiveStringMap(options.asJava) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ - import df.sparkSession.sessionState.analyzer.catalogManager + val catalogManager = df.sparkSession.sessionState.catalogManager mode match { case SaveMode.Append | SaveMode.Overwrite => val (table, catalogOpt, ident) = provider match { @@ -267,7 +267,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val catalog = CatalogV2Util.getTableProviderCatalog( supportsExtract, catalogManager, dsOptions) - (catalog.loadTable(ident), Some(catalog), Seq(ident)) + (catalog.loadTable(ident), catalogManager.catalogIdentifier(catalog), Seq(ident)) case tableProvider: TableProvider => val t = tableProvider.getTable(dsOptions) if (t.supports(BATCH_WRITE)) { @@ -280,12 +280,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } - val relation = DataSourceV2Relation.create( - table, - catalogOpt.map(catalogManager.catalogIdentifier(_)).flatten, - ident, - dsOptions - ) + val relation = DataSourceV2Relation.create(table, catalogOpt, ident, dsOptions) checkPartitioningMatchesV2Table(table) if (mode == SaveMode.Append) { runCommand(df.sparkSession, "save") { From 9fd93023dec3a1c0508d06355a2d49635ce13aa6 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 9 Jan 2020 17:40:26 -0800 Subject: [PATCH 08/23] remove stale comment --- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 66872b7e703e..3b49a155ec0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -227,7 +227,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ table match { case _: SupportsRead if table.supports(BATCH_READ) => - // TODO: Pass the PathIdentifiers as the list to V2Relation once that's implemented. Dataset.ofRows( sparkSession, DataSourceV2Relation.create(table, catalogOpt, ident, dsOptions)) From 4150d70d3fe17359ab70365be9fce3b417e21005 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 9 Jan 2020 17:53:12 -0800 Subject: [PATCH 09/23] nit --- .../org/apache/spark/sql/DataFrameReader.scala | 4 ++-- .../org/apache/spark/sql/DataFrameWriter.scala | 2 +- .../connector/TableCapabilityCheckSuite.scala | 18 +++++++++--------- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 3b49a155ec0f..7c9d49316f08 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -220,8 +220,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { (catalog.loadTable(ident), catalogManager.catalogIdentifier(catalog), Seq(ident)) case _ => userSpecifiedSchema match { - case Some(schema) => (provider.getTable(dsOptions, schema), None, Seq.empty) - case _ => (provider.getTable(dsOptions), None, Seq.empty) + case Some(schema) => (provider.getTable(dsOptions, schema), None, Nil) + case _ => (provider.getTable(dsOptions), None, Nil) } } import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index a36a2aa0edca..45f597fa9482 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -271,7 +271,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case tableProvider: TableProvider => val t = tableProvider.getTable(dsOptions) if (t.supports(BATCH_WRITE)) { - (t, None, Seq.empty) + (t, None, Nil) } else { // Streaming also uses the data source V2 API. So it may be that the data source // implements v2, but has no v2 implementation for batch writes. In that case, we diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index 3cb5b7eba075..967ad22abb8a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -56,7 +56,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { TableCapabilityCheck.apply(DataSourceV2Relation.create( CapabilityTable(), None, - Seq.empty, + Nil, CaseInsensitiveStringMap.empty)) } assert(e.message.contains("does not support batch scan")) @@ -93,7 +93,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { DataSourceV2Relation.create( CapabilityTable(), None, - Seq.empty, + Nil, CaseInsensitiveStringMap.empty), TestRelation) @@ -110,7 +110,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { DataSourceV2Relation.create( CapabilityTable(write), None, - Seq.empty, + Nil, CaseInsensitiveStringMap.empty), TestRelation) @@ -129,7 +129,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { DataSourceV2Relation.create( table, None, - Seq.empty, + Nil, CaseInsensitiveStringMap.empty), TestRelation, Literal(true)) @@ -152,7 +152,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { DataSourceV2Relation.create( table, None, - Seq.empty, + Nil, CaseInsensitiveStringMap.empty), TestRelation, Literal(true)) @@ -170,7 +170,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { DataSourceV2Relation.create( table, None, - Seq.empty, + Nil, CaseInsensitiveStringMap.empty), TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) @@ -190,7 +190,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { DataSourceV2Relation.create( table, None, - Seq.empty, + Nil, CaseInsensitiveStringMap.empty), TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) @@ -208,7 +208,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { DataSourceV2Relation.create( table, None, - Seq.empty, + Nil, CaseInsensitiveStringMap.empty), TestRelation) @@ -226,7 +226,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { DataSourceV2Relation.create( table, None, - Seq.empty, + Nil, CaseInsensitiveStringMap.empty), TestRelation) From 2bd689e95a5495347ed0d2f3e73f7b35f8dfe23b Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Fri, 10 Jan 2020 11:04:15 -0800 Subject: [PATCH 10/23] analyzer tests --- .../command/PlanResolutionSuite.scala | 70 +++++++++++++++++-- 1 file changed, 66 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 3f95e92e95b6..c7e11bff493a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -26,11 +26,11 @@ import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolveInlineTables, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedV2Relation} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{EqualTo, InSubquery, IntegerLiteral, ListQuery, StringLiteral} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeTable, DropTable, InsertAction, LogicalPlan, MergeIntoTable, OneRowRelation, Project, SubqueryAlias, UpdateAction, UpdateTable} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeTable, DropTable, InsertAction, InsertIntoStatement, LogicalPlan, MergeIntoTable, OneRowRelation, Project, ShowTableProperties, SubqueryAlias, UpdateAction, UpdateTable} import org.apache.spark.sql.connector.InMemoryTableProvider import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.execution.datasources.CreateTable @@ -106,10 +106,15 @@ class PlanResolutionSuite extends AnalysisTest { invocation.getArgument[String](0) match { case "testcat" => testCat + case CatalogManager.SESSION_CATALOG_NAME => + v2SessionCatalog case name => throw new CatalogNotFoundException(s"No such catalog: $name") } }) + when(manager.catalogIdentifier(testCat)).thenReturn(Some("testcat")) + when(manager.catalogIdentifier(v2SessionCatalog)) + .thenReturn(Some(CatalogManager.SESSION_CATALOG_NAME)) when(manager.currentCatalog).thenReturn(testCat) when(manager.currentNamespace).thenReturn(Array.empty[String]) when(manager.v1SessionCatalog).thenReturn(v1SessionCatalog) @@ -126,7 +131,10 @@ class PlanResolutionSuite extends AnalysisTest { throw new CatalogNotFoundException(s"No such catalog: $name") } }) + when(manager.catalogIdentifier(testCat)).thenReturn(Some("testcat")) when(manager.currentCatalog).thenReturn(v2SessionCatalog) + when(manager.catalogIdentifier(v2SessionCatalog)) + .thenReturn(Some(CatalogManager.SESSION_CATALOG_NAME)) when(manager.v1SessionCatalog).thenReturn(v1SessionCatalog) manager } @@ -142,8 +150,9 @@ class PlanResolutionSuite extends AnalysisTest { CTESubstitution, new ResolveCatalogs(catalogManager), new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), - analyzer.ResolveTables, - analyzer.ResolveRelations) + ResolveInlineTables(conf), + analyzer.ResolveRelations, + analyzer.ResolveTables) rules.foldLeft(parsePlan(query)) { case (plan, rule) => rule.apply(plan) } @@ -1066,6 +1075,59 @@ class PlanResolutionSuite extends AnalysisTest { } } + val DSV2ResolutionTests = { + val v2SessionCatalogTable = s"${CatalogManager.SESSION_CATALOG_NAME}.v2Table" + Seq( + ("ALTER TABLE testcat.tab ALTER COLUMN i TYPE bigint", false), + ("ALTER TABLE tab ALTER COLUMN i TYPE bigint", false), + (s"ALTER TABLE $v2SessionCatalogTable ALTER COLUMN i TYPE bigint", true), + ("INSERT INTO TABLE tab VALUES (1)", false), + ("INSERT INTO TABLE testcat.tab VALUES (1)", false), + (s"INSERT INTO TABLE $v2SessionCatalogTable VALUES (1)", true), + ("DESC TABLE tab", false), + ("DESC TABLE testcat.tab", false), + (s"DESC TABLE $v2SessionCatalogTable", true), + ("SHOW TBLPROPERTIES tab", false), + ("SHOW TBLPROPERTIES testcat.tab", false), + (s"SHOW TBLPROPERTIES $v2SessionCatalogTable", true), + ("SELECT * from tab", false), + ("SELECT * from testcat.tab", false), + (s"SELECT * from ${CatalogManager.SESSION_CATALOG_NAME}.v2Table", true) + ) + } + + DSV2ResolutionTests.foreach { case (sql, isSessionCatlog) => + test(s"Data source V2 relation resolution '$sql'") { + val parsed = parseAndResolve(sql, withDefault = true) + val catlogIdent = if (isSessionCatlog) CatalogManager.SESSION_CATALOG_NAME else "testcat" + val tableIdent = if (isSessionCatlog) "v2Table" else "tab" + parsed match { + case AlterTable(_, _, r: DataSourceV2Relation, _) => + assert(r.catalogIdentifier.exists(_ == catlogIdent)) + assert(r.identifiers.size == 1) + assert(r.identifiers.head.name() == tableIdent) + case Project(_, r: DataSourceV2Relation) => + assert(r.catalogIdentifier.exists(_ == catlogIdent)) + assert(r.identifiers.size == 1) + assert(r.identifiers.head.name() == tableIdent) + case InsertIntoStatement(r: DataSourceV2Relation, _, _, _, _) => + assert(r.catalogIdentifier.exists(_ == catlogIdent)) + assert(r.identifiers.size == 1) + assert(r.identifiers.head.name() == tableIdent) + case DescribeTable(r: DataSourceV2Relation, _) => + assert(r.catalogIdentifier.exists(_ == catlogIdent)) + assert(r.identifiers.size == 1) + assert(r.identifiers.head.name() == tableIdent) + case ShowTableProperties(r: DataSourceV2Relation, _) => + assert(r.catalogIdentifier.exists(_ == catlogIdent)) + assert(r.identifiers.size == 1) + assert(r.identifiers.head.name() == tableIdent) + case ShowTablePropertiesCommand(t: TableIdentifier, _) => + assert(t.identifier == tableIdent) + } + } + } + test("MERGE INTO TABLE") { Seq(("v2Table", "v2Table1"), ("testcat.tab", "testcat.tab1")).foreach { case(target, source) => From 442a8732305f101625024fa7a958f70dfd2b53a6 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Fri, 10 Jan 2020 11:45:37 -0800 Subject: [PATCH 11/23] catalog manager tests --- .../connector/catalog/CatalogManager.scala | 8 ++++++++ .../catalog/CatalogManagerSuite.scala | 20 +++++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index b00fa80760ef..43c87ceb4f2f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -48,6 +48,7 @@ class CatalogManager( // CatalogPlugin's name as it might be different from the catalog name depending on // implementation. private val catalogIdentifiers = mutable.HashMap.empty[CatalogPlugin, String] + catalogIdentifiers(defaultSessionCatalog) = SESSION_CATALOG_NAME def catalog(name: String): CatalogPlugin = synchronized { if (name.equalsIgnoreCase(SESSION_CATALOG_NAME)) { @@ -61,6 +62,13 @@ class CatalogManager( } } + /** + * Returns the identifier string for the given catalog + * + * @param catalog catalog to look up + * @return string identifier for the given catalog. If the catalog hasn't be registered return + * None + */ def catalogIdentifier(catalog: CatalogPlugin): Option[String] = synchronized { catalogIdentifiers.get(catalog) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala index fc7b2e720b7b..0eef7a663117 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala @@ -110,6 +110,26 @@ class CatalogManagerSuite extends SparkFunSuite { catalogManager.setCurrentNamespace(Array("test2")) assert(v1SessionCatalog.getCurrentDatabase == "default") } + + test("Catalog manager should be able to return the string identifier for registered catalog") { + val conf = new SQLConf + conf.setConfString("spark.sql.catalog.dummy", classOf[DummyCatalog].getName) + conf.setConfString(s"spark.sql.catalog.${CatalogManager.SESSION_CATALOG_NAME}", + classOf[DummyCatalog].getName) + val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog, createSessionCatalog(conf)) + + assert(catalogManager.catalogIdentifier(catalogManager.catalog("dummy")) == Some("dummy")) + assert(catalogManager.catalogIdentifier(catalogManager.v2SessionCatalog) == + Some(CatalogManager.SESSION_CATALOG_NAME)) + } + + test("Catalog manager should be able to return the string identifier for default catalog " + + "if no custom session catalog is provided") { + val conf = new SQLConf + val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog, createSessionCatalog(conf)) + assert(catalogManager.catalogIdentifier(catalogManager.v2SessionCatalog) == + Some(CatalogManager.SESSION_CATALOG_NAME)) + } } class DummyCatalog extends SupportsNamespaces { From ef4e7256264bf1687b77bd093180c8f88db98721 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Fri, 10 Jan 2020 12:47:16 -0800 Subject: [PATCH 12/23] V2UtilSuite --- .../catalog/CatalogV2UtilSuite.scala | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala new file mode 100644 index 000000000000..3632458959ee --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala @@ -0,0 +1,40 @@ +/* + * 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.connector.catalog + +import org.mockito.Mockito.{mock, when} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.types.StructType + +class CatalogV2UtilSuite extends SparkFunSuite { + test("Load relation should encode the identifiers for V2Relations") { + val testCatalog = mock(classOf[TableCatalog]) + val ident = mock(classOf[Identifier]) + val table = mock(classOf[Table]) + when(table.schema()).thenReturn(mock(classOf[StructType])) + when(testCatalog.loadTable(ident)).thenReturn(table) + val r = CatalogV2Util.loadRelation(testCatalog, Some("dummy"), ident) + assert(r.isDefined) + assert(r.get.isInstanceOf[DataSourceV2Relation]) + val v2Relation = r.get.asInstanceOf[DataSourceV2Relation] + assert(v2Relation.catalogIdentifier == Some("dummy")) + assert(v2Relation.identifiers == Seq(ident)) + } +} From 2730a49d07bbab8f21c8ce89647082799280f6c2 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Fri, 10 Jan 2020 17:34:24 -0800 Subject: [PATCH 13/23] add test for data frame reader --- .../sql/connector/SupportsCatalogOptionsSuite.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala index 0148bb07ee96..969a5232a5e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.connector.catalog.{Identifier, SupportsCatalogOptions, TableCatalog} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{LongType, StructType} @@ -196,8 +197,18 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with assert(e.getMessage.contains("not support user specified schema")) } + test("DataFrameReader create v2Relation with identifiers") { + sql(s"create table $catalogName.t1 (id bigint) using $format") + val df = load("t1", Some(catalogName)) + assert(df.logicalPlan.isInstanceOf[DataSourceV2Relation]) + val v2Relation = df.logicalPlan.asInstanceOf[DataSourceV2Relation] + assert(v2Relation.identifiers.length == 1) + assert(v2Relation.identifiers.head.name() == "t1") + assert(v2Relation.catalogIdentifier.exists(_ == catalogName)) + } + private def load(name: String, catalogOpt: Option[String]): DataFrame = { - val dfr = spark.read.format(format).option("name", "t1") + val dfr = spark.read.format(format).option("name", name) catalogOpt.foreach(cName => dfr.option("catalog", cName)) dfr.load() } From 77661296529a515403cbdae254f0013a324990e7 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Fri, 10 Jan 2020 17:54:05 -0800 Subject: [PATCH 14/23] nits --- .../datasources/v2/DataSourceV2Relation.scala | 5 +- .../apache/spark/sql/DataFrameWriter.scala | 9 ++-- .../connector/TableCapabilityCheckSuite.scala | 50 ++++--------------- 3 files changed, 16 insertions(+), 48 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index fd984359465f..c95ad4231572 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -33,9 +33,10 @@ import org.apache.spark.util.Utils * * @param table The table that this relation represents. * @param output the output attributes of this relation - * @param catalogIdentifier the string identifier for the catalog + * @param catalogIdentifier the string identifier for the catalog. None if no catalog is specified * @param identifiers the identifiers for the v2 relation. For multipath dataframe, there could be - * more than one identifier + * more than one identifier or Nil if a V2 relation is instantiated using + * options * @param options The options for this table operation. It's used to create fresh [[ScanBuilder]] * and [[WriteBuilder]]. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 45f597fa9482..bc03a1af228f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -561,12 +561,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case (SaveMode.Append, Some(table)) => checkPartitioningMatchesV2Table(table) - AppendData.byName( - DataSourceV2Relation.create( - table, - catalogManager.catalogIdentifier(catalog), - Seq(ident)), - df.logicalPlan, extraOptions.toMap) + val v2Relation = + DataSourceV2Relation.create(table, catalogManager.catalogIdentifier(catalog), Seq(ident)) + AppendData.byName(v2Relation, df.logicalPlan, extraOptions.toMap) case (SaveMode.Overwrite, _) => ReplaceTableAsSelect( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index 967ad22abb8a..8d11524a6e85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { + private val emptyMap = CaseInsensitiveStringMap.empty private def createStreamingRelation(table: Table, v1Relation: Option[StreamingRelation]) = { StreamingRelationV2( TestTableProvider, @@ -90,11 +91,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("AppendData: check missing capabilities") { val plan = AppendData.byName( - DataSourceV2Relation.create( - CapabilityTable(), - None, - Nil, - CaseInsensitiveStringMap.empty), + DataSourceV2Relation.create(CapabilityTable(), None, Nil, emptyMap), TestRelation) val exc = intercept[AnalysisException]{ @@ -107,11 +104,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("AppendData: check correct capabilities") { Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val plan = AppendData.byName( - DataSourceV2Relation.create( - CapabilityTable(write), - None, - Nil, - CaseInsensitiveStringMap.empty), + DataSourceV2Relation.create(CapabilityTable(write), None, Nil, emptyMap), TestRelation) TableCapabilityCheck.apply(plan) @@ -126,11 +119,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create( - table, - None, - Nil, - CaseInsensitiveStringMap.empty), + DataSourceV2Relation.create(table, None, Nil, emptyMap), TestRelation, Literal(true)) @@ -149,11 +138,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(V1_BATCH_WRITE, OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create( - table, - None, - Nil, - CaseInsensitiveStringMap.empty), TestRelation, + DataSourceV2Relation.create(table, None, Nil, emptyMap), + TestRelation, Literal(true)) TableCapabilityCheck.apply(plan) @@ -167,11 +153,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create( - table, - None, - Nil, - CaseInsensitiveStringMap.empty), + DataSourceV2Relation.create(table, None, Nil, emptyMap), TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) @@ -187,11 +169,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val table = CapabilityTable(write, OVERWRITE_BY_FILTER) val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create( - table, - None, - Nil, - CaseInsensitiveStringMap.empty), + DataSourceV2Relation.create(table, None, Nil, emptyMap), TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) @@ -205,11 +183,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_DYNAMIC)).foreach { table => val plan = OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create( - table, - None, - Nil, - CaseInsensitiveStringMap.empty), + DataSourceV2Relation.create(table, None, Nil, emptyMap), TestRelation) val exc = intercept[AnalysisException] { @@ -223,11 +197,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("OverwritePartitionsDynamic: check correct capabilities") { val table = CapabilityTable(BATCH_WRITE, OVERWRITE_DYNAMIC) val plan = OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create( - table, - None, - Nil, - CaseInsensitiveStringMap.empty), + DataSourceV2Relation.create(table, None, Nil, emptyMap), TestRelation) TableCapabilityCheck.apply(plan) From eb54fa00609833151638ac0a4af2fdcd4ca26cb7 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Fri, 10 Jan 2020 18:04:30 -0800 Subject: [PATCH 15/23] comments and tests --- .../apache/spark/sql/connector/catalog/CatalogManager.scala | 2 +- .../spark/sql/connector/catalog/CatalogManagerSuite.scala | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index 43c87ceb4f2f..022149ee3920 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -46,7 +46,7 @@ class CatalogManager( private val catalogs = mutable.HashMap.empty[String, CatalogPlugin] // Map from catalog back to it's original name for easy name look up, we don't use the // CatalogPlugin's name as it might be different from the catalog name depending on - // implementation. + // implementation. Catalog name <-> CatalogPlugin instance is a 1:1 mapping. private val catalogIdentifiers = mutable.HashMap.empty[CatalogPlugin, String] catalogIdentifiers(defaultSessionCatalog) = SESSION_CATALOG_NAME diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala index 0eef7a663117..ebdda35cf31d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala @@ -114,11 +114,15 @@ class CatalogManagerSuite extends SparkFunSuite { test("Catalog manager should be able to return the string identifier for registered catalog") { val conf = new SQLConf conf.setConfString("spark.sql.catalog.dummy", classOf[DummyCatalog].getName) + conf.setConfString("spark.sql.catalog.dummy2", classOf[DummyCatalog].getName) conf.setConfString(s"spark.sql.catalog.${CatalogManager.SESSION_CATALOG_NAME}", classOf[DummyCatalog].getName) val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog, createSessionCatalog(conf)) assert(catalogManager.catalogIdentifier(catalogManager.catalog("dummy")) == Some("dummy")) + // Even though dummy and dummy2 use the same catalog implementation, we should still be able to + // resolve the name as the CatalogPlugin instance is different. + assert(catalogManager.catalogIdentifier(catalogManager.catalog("dummy2")) == Some("dummy2")) assert(catalogManager.catalogIdentifier(catalogManager.v2SessionCatalog) == Some(CatalogManager.SESSION_CATALOG_NAME)) } From c80a155610472029eb1e2adf988aba3d0cf54206 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Mon, 13 Jan 2020 18:27:57 -0800 Subject: [PATCH 16/23] dataframewriter tests --- .../SupportsCatalogOptionsSuite.scala | 76 +++++++++++++++++-- 1 file changed, 69 insertions(+), 7 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala index 969a5232a5e9..88f13c2313c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala @@ -26,14 +26,16 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression} import org.apache.spark.sql.connector.catalog.{Identifier, SupportsCatalogOptions, TableCatalog} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} +import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{LongType, StructType} -import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.{CaseInsensitiveStringMap, QueryExecutionListener} class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with BeforeAndAfter { @@ -197,14 +199,68 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with assert(e.getMessage.contains("not support user specified schema")) } - test("DataFrameReader create v2Relation with identifiers") { + test("DataFrameReader creates v2Relation with identifiers") { sql(s"create table $catalogName.t1 (id bigint) using $format") val df = load("t1", Some(catalogName)) - assert(df.logicalPlan.isInstanceOf[DataSourceV2Relation]) - val v2Relation = df.logicalPlan.asInstanceOf[DataSourceV2Relation] - assert(v2Relation.identifiers.length == 1) - assert(v2Relation.identifiers.head.name() == "t1") - assert(v2Relation.catalogIdentifier.exists(_ == catalogName)) + checkV2Identifiers(df.logicalPlan) + } + + test("DataFrameWriter creates v2Relation with identifiers") { + sql(s"create table $catalogName.t1 (id bigint) using $format") + + var plan: LogicalPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.analyzed + + } + override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {} + } + + spark.listenerManager.register(listener) + + try { + // Test append + save("t1", SaveMode.Append, Some(catalogName)) + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[AppendData]) + val appendRelation = plan.asInstanceOf[AppendData].table + checkV2Identifiers(appendRelation) + + // Test overwrite + save("t1", SaveMode.Overwrite, Some(catalogName)) + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[OverwriteByExpression]) + val overwriteRelation = plan.asInstanceOf[OverwriteByExpression].table + checkV2Identifiers(overwriteRelation) + + // Test insert + spark.range(10).write.format(format).insertInto(s"$catalogName.t1") + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[AppendData]) + val insertRelation = plan.asInstanceOf[AppendData].table + checkV2Identifiers(insertRelation) + + // Test saveAsTable append + spark.range(10).write.format(format).mode(SaveMode.Append).saveAsTable(s"$catalogName.t1") + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[AppendData]) + val saveAsTableRelation = plan.asInstanceOf[AppendData].table + checkV2Identifiers(saveAsTableRelation) + } finally { + spark.listenerManager.unregister(listener) + } + } + + private def checkV2Identifiers( + plan: LogicalPlan, + identifiers: Seq[String] = Seq("t1"), + catalogName: String = catalogName): Unit = { + assert(plan.isInstanceOf[DataSourceV2Relation]) + val v2 = plan.asInstanceOf[DataSourceV2Relation] + assert(v2.identifiers.length == identifiers.length) + assert(identifiers.forall(t => v2.identifiers.exists(_.name() == t))) + assert(v2.catalogIdentifier.exists(_ == catalogName)) } private def load(name: String, catalogOpt: Option[String]): DataFrame = { @@ -212,6 +268,12 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with catalogOpt.foreach(cName => dfr.option("catalog", cName)) dfr.load() } + + private def save(name: String, mode: SaveMode, catalogOpt: Option[String]): Unit = { + val df = spark.range(10) + df.write.format(format).option("name", name).option("catalog", catalogName) + .mode(mode).save() + } } class CatalogSupportingInMemoryTableProvider From 42bf8722f3634c0975db89af8dfcd3373a12e5f7 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Mon, 13 Jan 2020 18:52:37 -0800 Subject: [PATCH 17/23] DataFrameWriterV2 tests --- .../spark/sql/DataFrameWriterV2Suite.scala | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index ce0a5f21fd7e..64eb87496711 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -22,11 +22,15 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} import org.apache.spark.sql.connector.InMemoryTableCatalog import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.connector.expressions.{BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with BeforeAndAfter { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -51,6 +55,46 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.sessionState.conf.clear() } + test("DataFrameWriteV2 encode identifiers correctly") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + var plan: LogicalPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.analyzed + + } + override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {} + } + spark.listenerManager.register(listener) + + spark.table("source").writeTo("testcat.table_name").append() + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[AppendData]) + checkV2Identifiers(plan.asInstanceOf[AppendData].table) + + spark.table("source").writeTo("testcat.table_name").overwrite(lit(true)) + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[OverwriteByExpression]) + checkV2Identifiers(plan.asInstanceOf[OverwriteByExpression].table) + + spark.table("source").writeTo("testcat.table_name").overwritePartitions() + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[OverwritePartitionsDynamic]) + checkV2Identifiers(plan.asInstanceOf[OverwritePartitionsDynamic].table) + } + + private def checkV2Identifiers( + plan: LogicalPlan, + identifiers: Seq[String] = Seq("table_name"), + catalogName: String = "testcat"): Unit = { + assert(plan.isInstanceOf[DataSourceV2Relation]) + val v2 = plan.asInstanceOf[DataSourceV2Relation] + assert(v2.identifiers.length == identifiers.length) + assert(identifiers.forall(t => v2.identifiers.exists(_.name() == t))) + assert(v2.catalogIdentifier.exists(_ == catalogName)) + } + test("Append: basic append") { spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") From 904278f8f1e99e45c3a09fef8d957244232813bf Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 23 Jan 2020 00:30:03 -0800 Subject: [PATCH 18/23] final interface --- .../sql/catalyst/analysis/Analyzer.scala | 24 ++++--------- .../connector/catalog/CatalogManager.scala | 28 ++------------- .../sql/connector/catalog/CatalogV2Util.scala | 3 +- .../datasources/v2/DataSourceV2Relation.scala | 24 ++++++------- .../catalog/CatalogManagerSuite.scala | 24 ------------- .../catalog/CatalogV2UtilSuite.scala | 4 +-- .../apache/spark/sql/DataFrameReader.scala | 11 +++--- .../apache/spark/sql/DataFrameWriter.scala | 16 ++++----- .../apache/spark/sql/DataFrameWriterV2.scala | 36 +++++++++---------- 9 files changed, 50 insertions(+), 120 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index b93ab861d1d8..55abd2fd186f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -809,27 +809,22 @@ class Analyzer( .getOrElse(i) case desc @ DescribeTable(u: UnresolvedV2Relation, _) => - CatalogV2Util - .loadRelation(u.catalog, catalogManager.catalogIdentifier(u.catalog), u.tableName) + CatalogV2Util.loadRelation(u.catalog, u.tableName) .map(rel => desc.copy(table = rel)) .getOrElse(desc) case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) => - CatalogV2Util - .loadRelation(u.catalog, catalogManager.catalogIdentifier(u.catalog), u.tableName) + CatalogV2Util.loadRelation(u.catalog, u.tableName) .map(rel => alter.copy(table = rel)) .getOrElse(alter) case show @ ShowTableProperties(u: UnresolvedV2Relation, _) => - CatalogV2Util - .loadRelation(u.catalog, catalogManager.catalogIdentifier(u.catalog), u.tableName) + CatalogV2Util.loadRelation(u.catalog, u.tableName) .map(rel => show.copy(table = rel)) .getOrElse(show) case u: UnresolvedV2Relation => - CatalogV2Util - .loadRelation(u.catalog, catalogManager.catalogIdentifier(u.catalog), u.tableName) - .getOrElse(u) + CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u) } /** @@ -840,10 +835,7 @@ class Analyzer( case NonSessionCatalogAndIdentifier(catalog, ident) => CatalogV2Util.loadTable(catalog, ident) match { case Some(table) => - Some(DataSourceV2Relation.create( - table, - catalogManager.catalogIdentifier(catalog), - Seq(ident))) + Some(DataSourceV2Relation.create(table, catalog, ident)) case None => None } case _ => None @@ -924,11 +916,7 @@ class Analyzer( AnalysisContext.get.relationCache.getOrElseUpdate( key, v1SessionCatalog.getRelation(v1Table.v1Table)) case table => - DataSourceV2Relation.create( - table, - catalogManager.catalogIdentifier(catalog), - Seq(ident) - ) + DataSourceV2Relation.create(table, catalog, ident) } case _ => None } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index 022149ee3920..bbb7a182c6a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -44,35 +44,15 @@ class CatalogManager( import CatalogManager.SESSION_CATALOG_NAME private val catalogs = mutable.HashMap.empty[String, CatalogPlugin] - // Map from catalog back to it's original name for easy name look up, we don't use the - // CatalogPlugin's name as it might be different from the catalog name depending on - // implementation. Catalog name <-> CatalogPlugin instance is a 1:1 mapping. - private val catalogIdentifiers = mutable.HashMap.empty[CatalogPlugin, String] - catalogIdentifiers(defaultSessionCatalog) = SESSION_CATALOG_NAME def catalog(name: String): CatalogPlugin = synchronized { if (name.equalsIgnoreCase(SESSION_CATALOG_NAME)) { v2SessionCatalog } else { - catalogs.getOrElseUpdate(name, { - val catalog = Catalogs.load(name, conf) - catalogIdentifiers(catalog) = name - catalog - }) + catalogs.getOrElseUpdate(name, Catalogs.load(name, conf)) } } - /** - * Returns the identifier string for the given catalog - * - * @param catalog catalog to look up - * @return string identifier for the given catalog. If the catalog hasn't be registered return - * None - */ - def catalogIdentifier(catalog: CatalogPlugin): Option[String] = synchronized { - catalogIdentifiers.get(catalog) - } - def isCatalogRegistered(name: String): Boolean = { try { catalog(name) @@ -103,11 +83,7 @@ class CatalogManager( private[sql] def v2SessionCatalog: CatalogPlugin = { conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).map { customV2SessionCatalog => try { - catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, { - val catalog = loadV2SessionCatalog() - catalogIdentifiers(catalog) = SESSION_CATALOG_NAME - catalog - }) + catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) } catch { case NonFatal(_) => logError( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 308600922742..058d2a5475be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -258,9 +258,8 @@ private[sql] object CatalogV2Util { def loadRelation( catalog: CatalogPlugin, - catalogIdentifier: Option[String], ident: Identifier): Option[NamedRelation] = { - loadTable(catalog, ident).map(DataSourceV2Relation.create(_, catalogIdentifier, Seq(ident))) + loadTable(catalog, ident).map(DataSourceV2Relation.create(_, catalog, ident)) } def isSessionCatalog(catalog: CatalogPlugin): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index c95ad4231572..5afaf4e3b70e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -32,19 +32,17 @@ import org.apache.spark.util.Utils * A logical plan representing a data source v2 table. * * @param table The table that this relation represents. - * @param output the output attributes of this relation - * @param catalogIdentifier the string identifier for the catalog. None if no catalog is specified - * @param identifiers the identifiers for the v2 relation. For multipath dataframe, there could be - * more than one identifier or Nil if a V2 relation is instantiated using - * options + * @param output the output attributes of this relation. + * @param catalog catalogPlugin for the table. + * @param identifier the identifier for the table. * @param options The options for this table operation. It's used to create fresh [[ScanBuilder]] * and [[WriteBuilder]]. */ case class DataSourceV2Relation( table: Table, output: Seq[AttributeReference], - catalogIdentifier: Option[String], - identifiers: Seq[Identifier], + catalog: CatalogPlugin, + identifier: Identifier, options: CaseInsensitiveStringMap) extends LeafNode with MultiInstanceRelation with NamedRelation { @@ -146,18 +144,18 @@ case class StreamingDataSourceV2Relation( object DataSourceV2Relation { def create( table: Table, - catalogIdentifier: Option[String], - identifiers: Seq[Identifier], + catalog: CatalogPlugin, + identifiers: Identifier, options: CaseInsensitiveStringMap): DataSourceV2Relation = { val output = table.schema().toAttributes - DataSourceV2Relation(table, output, catalogIdentifier, identifiers, options) + DataSourceV2Relation(table, output, catalog, identifiers, options) } def create( table: Table, - catalogIdentifier: Option[String], - identifiers: Seq[Identifier]): DataSourceV2Relation = - create(table, catalogIdentifier, identifiers, CaseInsensitiveStringMap.empty) + catalog: CatalogPlugin, + identifier: Identifier): DataSourceV2Relation = + create(table, catalog, identifier, CaseInsensitiveStringMap.empty) /** * This is used to transform data source v2 statistics to logical.Statistics. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala index ebdda35cf31d..fc7b2e720b7b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala @@ -110,30 +110,6 @@ class CatalogManagerSuite extends SparkFunSuite { catalogManager.setCurrentNamespace(Array("test2")) assert(v1SessionCatalog.getCurrentDatabase == "default") } - - test("Catalog manager should be able to return the string identifier for registered catalog") { - val conf = new SQLConf - conf.setConfString("spark.sql.catalog.dummy", classOf[DummyCatalog].getName) - conf.setConfString("spark.sql.catalog.dummy2", classOf[DummyCatalog].getName) - conf.setConfString(s"spark.sql.catalog.${CatalogManager.SESSION_CATALOG_NAME}", - classOf[DummyCatalog].getName) - val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog, createSessionCatalog(conf)) - - assert(catalogManager.catalogIdentifier(catalogManager.catalog("dummy")) == Some("dummy")) - // Even though dummy and dummy2 use the same catalog implementation, we should still be able to - // resolve the name as the CatalogPlugin instance is different. - assert(catalogManager.catalogIdentifier(catalogManager.catalog("dummy2")) == Some("dummy2")) - assert(catalogManager.catalogIdentifier(catalogManager.v2SessionCatalog) == - Some(CatalogManager.SESSION_CATALOG_NAME)) - } - - test("Catalog manager should be able to return the string identifier for default catalog " + - "if no custom session catalog is provided") { - val conf = new SQLConf - val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog, createSessionCatalog(conf)) - assert(catalogManager.catalogIdentifier(catalogManager.v2SessionCatalog) == - Some(CatalogManager.SESSION_CATALOG_NAME)) - } } class DummyCatalog extends SupportsNamespaces { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala index 3632458959ee..f9e1ba240c70 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala @@ -34,7 +34,7 @@ class CatalogV2UtilSuite extends SparkFunSuite { assert(r.isDefined) assert(r.get.isInstanceOf[DataSourceV2Relation]) val v2Relation = r.get.asInstanceOf[DataSourceV2Relation] - assert(v2Relation.catalogIdentifier == Some("dummy")) - assert(v2Relation.identifiers == Seq(ident)) + assert(v2Relation.catalog == testCatalog) + assert(v2Relation.identifier == ident) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 7c9d49316f08..e3cdfd1b8e1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -207,7 +207,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val finalOptions = sessionOptions ++ extraOptions.toMap ++ pathsOption val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava) - val (table, catalogOpt, ident) = provider match { + val (table, catalog, ident) = provider match { case _: SupportsCatalogOptions if userSpecifiedSchema.nonEmpty => throw new IllegalArgumentException( s"$source does not support user specified schema. Please don't specify the schema.") @@ -217,11 +217,12 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { hasCatalog, catalogManager, dsOptions) - (catalog.loadTable(ident), catalogManager.catalogIdentifier(catalog), Seq(ident)) + (catalog.loadTable(ident), catalog, ident) case _ => + // TODO: Non-catalog paths for DSV2 are currently not well defined. userSpecifiedSchema match { - case Some(schema) => (provider.getTable(dsOptions, schema), None, Nil) - case _ => (provider.getTable(dsOptions), None, Nil) + case Some(schema) => (provider.getTable(dsOptions, schema), null, null) + case _ => (provider.getTable(dsOptions), null, null) } } import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ @@ -229,7 +230,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { case _: SupportsRead if table.supports(BATCH_READ) => Dataset.ofRows( sparkSession, - DataSourceV2Relation.create(table, catalogOpt, ident, dsOptions)) + DataSourceV2Relation.create(table, catalog, ident, dsOptions)) case _ => loadV1Source(paths: _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index bc03a1af228f..f18732a02186 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -261,17 +261,17 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val catalogManager = df.sparkSession.sessionState.catalogManager mode match { case SaveMode.Append | SaveMode.Overwrite => - val (table, catalogOpt, ident) = provider match { + val (table, catalog, ident) = provider match { case supportsExtract: SupportsCatalogOptions => val ident = supportsExtract.extractIdentifier(dsOptions) val catalog = CatalogV2Util.getTableProviderCatalog( supportsExtract, catalogManager, dsOptions) - (catalog.loadTable(ident), catalogManager.catalogIdentifier(catalog), Seq(ident)) + (catalog.loadTable(ident), catalog, ident) case tableProvider: TableProvider => val t = tableProvider.getTable(dsOptions) if (t.supports(BATCH_WRITE)) { - (t, None, Nil) + (t, null, null) } else { // Streaming also uses the data source V2 API. So it may be that the data source // implements v2, but has no v2 implementation for batch writes. In that case, we @@ -280,7 +280,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } - val relation = DataSourceV2Relation.create(table, catalogOpt, ident, dsOptions) + val relation = DataSourceV2Relation.create(table, catalog, ident, dsOptions) checkPartitioningMatchesV2Table(table) if (mode == SaveMode.Append) { runCommand(df.sparkSession, "save") { @@ -412,14 +412,13 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } private def insertInto(catalog: CatalogPlugin, ident: Identifier): Unit = { - import df.sparkSession.sessionState.analyzer.catalogManager import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val table = catalog.asTableCatalog.loadTable(ident) match { case _: V1Table => return insertInto(TableIdentifier(ident.name(), ident.namespace().headOption)) case t => - DataSourceV2Relation.create(t, catalogManager.catalogIdentifier(catalog), Seq(ident)) + DataSourceV2Relation.create(t, catalog, ident) } val command = mode match { @@ -544,8 +543,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def saveAsTable(catalog: TableCatalog, ident: Identifier): Unit = { - import df.sparkSession.sessionState.analyzer.catalogManager - val tableOpt = try Option(catalog.loadTable(ident)) catch { case _: NoSuchTableException => None } @@ -561,8 +558,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case (SaveMode.Append, Some(table)) => checkPartitioningMatchesV2Table(table) - val v2Relation = - DataSourceV2Relation.create(table, catalogManager.catalogIdentifier(catalog), Seq(ident)) + val v2Relation = DataSourceV2Relation.create(table, catalog, ident) AppendData.byName(v2Relation, df.logicalPlan, extraOptions.toMap) case (SaveMode.Overwrite, _) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index 52357fb1abd1..272c4dd1956d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -24,7 +24,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.expressions.{Attribute, Bucket, Days, Hours, Literal, Months, Years} import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} -import org.apache.spark.sql.connector.catalog.{CatalogManager, TableCatalog} +import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference, Transform} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation @@ -51,14 +51,10 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table) - private val (catalog, catalogIdentifier, tableIdentifier) = { - import df.sparkSession.sessionState.analyzer.{NonSessionCatalogAndIdentifier, SessionCatalogAndIdentifier} - tableName match { - case NonSessionCatalogAndIdentifier(catalog, identifier) => - (catalog.asTableCatalog, tableName.headOption, identifier) - case SessionCatalogAndIdentifier(catalog, identifier) => - (catalog.asTableCatalog, Some(CatalogManager.SESSION_CATALOG_NAME), identifier) - } + private val (catalog, identifier) = { + val CatalogAndIdentifier(catalog, identifier) = tableName + + (catalog.asTableCatalog, identifier) } private val logicalPlan = df.queryExecution.logical @@ -131,7 +127,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) runCommand("create") { CreateTableAsSelect( catalog, - tableIdentifier, + identifier, partitioning.getOrElse(Seq.empty), logicalPlan, properties = provider.map(p => properties + (TableCatalog.PROP_PROVIDER -> p)) @@ -161,13 +157,13 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) */ @throws(classOf[NoSuchTableException]) def append(): Unit = { - val append = loadTable(catalog, tableIdentifier) match { + val append = loadTable(catalog, identifier) match { case Some(t) => AppendData.byName( - DataSourceV2Relation.create(t, catalogIdentifier, Seq(tableIdentifier)), + DataSourceV2Relation.create(t, catalog, identifier), logicalPlan, options.toMap) case _ => - throw new NoSuchTableException(tableIdentifier) + throw new NoSuchTableException(identifier) } runCommand("append")(append) @@ -185,13 +181,13 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) */ @throws(classOf[NoSuchTableException]) def overwrite(condition: Column): Unit = { - val overwrite = loadTable(catalog, tableIdentifier) match { + val overwrite = loadTable(catalog, identifier) match { case Some(t) => OverwriteByExpression.byName( - DataSourceV2Relation.create(t, catalogIdentifier, Seq(tableIdentifier)), + DataSourceV2Relation.create(t, catalog, identifier), logicalPlan, condition.expr, options.toMap) case _ => - throw new NoSuchTableException(tableIdentifier) + throw new NoSuchTableException(identifier) } runCommand("overwrite")(overwrite) @@ -212,13 +208,13 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) */ @throws(classOf[NoSuchTableException]) def overwritePartitions(): Unit = { - val dynamicOverwrite = loadTable(catalog, tableIdentifier) match { + val dynamicOverwrite = loadTable(catalog, identifier) match { case Some(t) => OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(t, catalogIdentifier, Seq(tableIdentifier)), + DataSourceV2Relation.create(t, catalog, identifier), logicalPlan, options.toMap) case _ => - throw new NoSuchTableException(tableIdentifier) + throw new NoSuchTableException(identifier) } runCommand("overwritePartitions")(dynamicOverwrite) @@ -238,7 +234,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) runCommand("replace") { ReplaceTableAsSelect( catalog, - tableIdentifier, + identifier, partitioning.getOrElse(Seq.empty), logicalPlan, properties = provider.map(p => properties + ("provider" -> p)).getOrElse(properties).toMap, From 68c66f5f3624ae72baba5559fa75e9c3173dd1d5 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 23 Jan 2020 01:00:58 -0800 Subject: [PATCH 19/23] fix tests --- .../catalog/CatalogV2UtilSuite.scala | 2 +- .../spark/sql/DataFrameWriterV2Suite.scala | 9 +++-- .../SupportsCatalogOptionsSuite.scala | 15 ++++----- .../connector/TableCapabilityCheckSuite.scala | 28 +++++++++------- .../command/PlanResolutionSuite.scala | 33 +++++++------------ 5 files changed, 39 insertions(+), 48 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala index f9e1ba240c70..bb1295768288 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala @@ -30,7 +30,7 @@ class CatalogV2UtilSuite extends SparkFunSuite { val table = mock(classOf[Table]) when(table.schema()).thenReturn(mock(classOf[StructType])) when(testCatalog.loadTable(ident)).thenReturn(table) - val r = CatalogV2Util.loadRelation(testCatalog, Some("dummy"), ident) + val r = CatalogV2Util.loadRelation(testCatalog, ident) assert(r.isDefined) assert(r.get.isInstanceOf[DataSourceV2Relation]) val v2Relation = r.get.asInstanceOf[DataSourceV2Relation] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index 64eb87496711..b7eb0a1b2afb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -86,13 +86,12 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo private def checkV2Identifiers( plan: LogicalPlan, - identifiers: Seq[String] = Seq("table_name"), - catalogName: String = "testcat"): Unit = { + identifier: String = "table_name", + catalogPlugin: TableCatalog = catalog("testcat")): Unit = { assert(plan.isInstanceOf[DataSourceV2Relation]) val v2 = plan.asInstanceOf[DataSourceV2Relation] - assert(v2.identifiers.length == identifiers.length) - assert(identifiers.forall(t => v2.identifiers.exists(_.name() == t))) - assert(v2.catalogIdentifier.exists(_ == catalogName)) + assert(v2.identifier.name() == identifier) + assert(v2.catalog == catalogPlugin) } test("Append: basic append") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala index 88f13c2313c9..00a7d491f93e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala @@ -254,13 +254,12 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with private def checkV2Identifiers( plan: LogicalPlan, - identifiers: Seq[String] = Seq("t1"), - catalogName: String = catalogName): Unit = { + identifier: String = "t1", + catalogPlugin: TableCatalog = catalog(catalogName)): Unit = { assert(plan.isInstanceOf[DataSourceV2Relation]) val v2 = plan.asInstanceOf[DataSourceV2Relation] - assert(v2.identifiers.length == identifiers.length) - assert(identifiers.forall(t => v2.identifiers.exists(_.name() == t))) - assert(v2.catalogIdentifier.exists(_ == catalogName)) + assert(v2.identifier.name() == identifier) + assert(v2.catalog == catalogPlugin) } private def load(name: String, catalogOpt: Option[String]): DataFrame = { @@ -270,9 +269,9 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with } private def save(name: String, mode: SaveMode, catalogOpt: Option[String]): Unit = { - val df = spark.range(10) - df.write.format(format).option("name", name).option("catalog", catalogName) - .mode(mode).save() + val df = spark.range(10).write.format(format).option("name", name) + catalogOpt.foreach(cName => df.option("catalog", catalogName)) + df.mode(mode).save() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index 8d11524a6e85..f3969a981e6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -21,11 +21,13 @@ import java.util import scala.collection.JavaConverters._ +import org.scalatestplus.mockito.MockitoSugar + import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.analysis.{AnalysisSuite, NamedRelation} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.connector.catalog.{Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, Table, TableCapability, TableProvider} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, TableCapabilityCheck} @@ -35,9 +37,11 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap -class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { +class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession with MockitoSugar { private val emptyMap = CaseInsensitiveStringMap.empty + private val mockCatalog = mock[CatalogPlugin] + private val mockIdent = mock[Identifier] private def createStreamingRelation(table: Table, v1Relation: Option[StreamingRelation]) = { StreamingRelationV2( TestTableProvider, @@ -56,8 +60,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { val e = intercept[AnalysisException] { TableCapabilityCheck.apply(DataSourceV2Relation.create( CapabilityTable(), - None, - Nil, + mockCatalog, + mockIdent, CaseInsensitiveStringMap.empty)) } assert(e.message.contains("does not support batch scan")) @@ -91,7 +95,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("AppendData: check missing capabilities") { val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(), None, Nil, emptyMap), + DataSourceV2Relation.create(CapabilityTable(), mockCatalog, mockIdent, emptyMap), TestRelation) val exc = intercept[AnalysisException]{ @@ -104,7 +108,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("AppendData: check correct capabilities") { Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(write), None, Nil, emptyMap), + DataSourceV2Relation.create(CapabilityTable(write), mockCatalog, mockIdent, emptyMap), TestRelation) TableCapabilityCheck.apply(plan) @@ -119,7 +123,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, None, Nil, emptyMap), + DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), TestRelation, Literal(true)) @@ -138,7 +142,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(V1_BATCH_WRITE, OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, None, Nil, emptyMap), + DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), TestRelation, Literal(true)) @@ -153,7 +157,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, None, Nil, emptyMap), + DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) @@ -169,7 +173,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val table = CapabilityTable(write, OVERWRITE_BY_FILTER) val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, None, Nil, emptyMap), + DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) @@ -183,7 +187,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_DYNAMIC)).foreach { table => val plan = OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(table, None, Nil, emptyMap), + DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), TestRelation) val exc = intercept[AnalysisException] { @@ -197,7 +201,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("OverwritePartitionsDynamic: check correct capabilities") { val table = CapabilityTable(BATCH_WRITE, OVERWRITE_DYNAMIC) val plan = OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(table, None, Nil, emptyMap), + DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), TestRelation) TableCapabilityCheck.apply(plan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index c7e11bff493a..a8e4f8dda1b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -112,9 +112,6 @@ class PlanResolutionSuite extends AnalysisTest { throw new CatalogNotFoundException(s"No such catalog: $name") } }) - when(manager.catalogIdentifier(testCat)).thenReturn(Some("testcat")) - when(manager.catalogIdentifier(v2SessionCatalog)) - .thenReturn(Some(CatalogManager.SESSION_CATALOG_NAME)) when(manager.currentCatalog).thenReturn(testCat) when(manager.currentNamespace).thenReturn(Array.empty[String]) when(manager.v1SessionCatalog).thenReturn(v1SessionCatalog) @@ -131,10 +128,7 @@ class PlanResolutionSuite extends AnalysisTest { throw new CatalogNotFoundException(s"No such catalog: $name") } }) - when(manager.catalogIdentifier(testCat)).thenReturn(Some("testcat")) when(manager.currentCatalog).thenReturn(v2SessionCatalog) - when(manager.catalogIdentifier(v2SessionCatalog)) - .thenReturn(Some(CatalogManager.SESSION_CATALOG_NAME)) when(manager.v1SessionCatalog).thenReturn(v1SessionCatalog) manager } @@ -1099,29 +1093,24 @@ class PlanResolutionSuite extends AnalysisTest { DSV2ResolutionTests.foreach { case (sql, isSessionCatlog) => test(s"Data source V2 relation resolution '$sql'") { val parsed = parseAndResolve(sql, withDefault = true) - val catlogIdent = if (isSessionCatlog) CatalogManager.SESSION_CATALOG_NAME else "testcat" + val catlogIdent = if (isSessionCatlog) v2SessionCatalog else testCat val tableIdent = if (isSessionCatlog) "v2Table" else "tab" parsed match { case AlterTable(_, _, r: DataSourceV2Relation, _) => - assert(r.catalogIdentifier.exists(_ == catlogIdent)) - assert(r.identifiers.size == 1) - assert(r.identifiers.head.name() == tableIdent) + assert(r.catalog == catlogIdent) + assert(r.identifier.name() == tableIdent) case Project(_, r: DataSourceV2Relation) => - assert(r.catalogIdentifier.exists(_ == catlogIdent)) - assert(r.identifiers.size == 1) - assert(r.identifiers.head.name() == tableIdent) + assert(r.catalog == catlogIdent) + assert(r.identifier.name() == tableIdent) case InsertIntoStatement(r: DataSourceV2Relation, _, _, _, _) => - assert(r.catalogIdentifier.exists(_ == catlogIdent)) - assert(r.identifiers.size == 1) - assert(r.identifiers.head.name() == tableIdent) + assert(r.catalog == catlogIdent) + assert(r.identifier.name() == tableIdent) case DescribeTable(r: DataSourceV2Relation, _) => - assert(r.catalogIdentifier.exists(_ == catlogIdent)) - assert(r.identifiers.size == 1) - assert(r.identifiers.head.name() == tableIdent) + assert(r.catalog == catlogIdent) + assert(r.identifier.name() == tableIdent) case ShowTableProperties(r: DataSourceV2Relation, _) => - assert(r.catalogIdentifier.exists(_ == catlogIdent)) - assert(r.identifiers.size == 1) - assert(r.identifiers.head.name() == tableIdent) + assert(r.catalog == catlogIdent) + assert(r.identifier.name() == tableIdent) case ShowTablePropertiesCommand(t: TableIdentifier, _) => assert(t.identifier == tableIdent) } From 713afc913d9ce4950241a43c0004a5a2f676b0ee Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 23 Jan 2020 22:29:06 -0800 Subject: [PATCH 20/23] use option instead of null --- .../sql/catalyst/analysis/Analyzer.scala | 4 +-- .../sql/connector/catalog/CatalogV2Util.scala | 2 +- .../datasources/v2/DataSourceV2Relation.scala | 18 ++++++------- .../apache/spark/sql/DataFrameReader.scala | 6 ++--- .../apache/spark/sql/DataFrameWriter.scala | 8 +++--- .../apache/spark/sql/DataFrameWriterV2.scala | 6 ++--- .../connector/TableCapabilityCheckSuite.scala | 26 ++++++++----------- 7 files changed, 33 insertions(+), 37 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index c862d0ee6839..4cb73cb6977e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -832,7 +832,7 @@ class Analyzer( case NonSessionCatalogAndIdentifier(catalog, ident) => CatalogV2Util.loadTable(catalog, ident) match { case Some(table) => - Some(DataSourceV2Relation.create(table, catalog, ident)) + Some(DataSourceV2Relation.create(table, Some(catalog), Some(ident))) case None => None } case _ => None @@ -924,7 +924,7 @@ class Analyzer( AnalysisContext.get.relationCache.getOrElseUpdate( key, v1SessionCatalog.getRelation(v1Table.v1Table)) case table => - DataSourceV2Relation.create(table, catalog, ident) + DataSourceV2Relation.create(table, Some(catalog), Some(ident)) } case _ => None } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index b496f8f091fe..bff1134621e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -259,7 +259,7 @@ private[sql] object CatalogV2Util { def loadRelation( catalog: CatalogPlugin, ident: Identifier): Option[NamedRelation] = { - loadTable(catalog, ident).map(DataSourceV2Relation.create(_, catalog, ident)) + loadTable(catalog, ident).map(DataSourceV2Relation.create(_, Some(catalog), Some(ident))) } def isSessionCatalog(catalog: CatalogPlugin): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 5afaf4e3b70e..45d89498f5ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -33,16 +33,16 @@ import org.apache.spark.util.Utils * * @param table The table that this relation represents. * @param output the output attributes of this relation. - * @param catalog catalogPlugin for the table. - * @param identifier the identifier for the table. + * @param catalog catalogPlugin for the table. None if no catalog is specified. + * @param identifier the identifier for the table. None if no identifier is defined. * @param options The options for this table operation. It's used to create fresh [[ScanBuilder]] * and [[WriteBuilder]]. */ case class DataSourceV2Relation( table: Table, output: Seq[AttributeReference], - catalog: CatalogPlugin, - identifier: Identifier, + catalog: Option[CatalogPlugin], + identifier: Option[Identifier], options: CaseInsensitiveStringMap) extends LeafNode with MultiInstanceRelation with NamedRelation { @@ -144,17 +144,17 @@ case class StreamingDataSourceV2Relation( object DataSourceV2Relation { def create( table: Table, - catalog: CatalogPlugin, - identifiers: Identifier, + catalog: Option[CatalogPlugin], + identifier: Option[Identifier], options: CaseInsensitiveStringMap): DataSourceV2Relation = { val output = table.schema().toAttributes - DataSourceV2Relation(table, output, catalog, identifiers, options) + DataSourceV2Relation(table, output, catalog, identifier, options) } def create( table: Table, - catalog: CatalogPlugin, - identifier: Identifier): DataSourceV2Relation = + catalog: Option[CatalogPlugin], + identifier: Option[Identifier]): DataSourceV2Relation = create(table, catalog, identifier, CaseInsensitiveStringMap.empty) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 3917991f8ede..b5d7bbca9064 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -217,12 +217,12 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { hasCatalog, catalogManager, dsOptions) - (catalog.loadTable(ident), catalog, ident) + (catalog.loadTable(ident), Some(catalog), Some(ident)) case _ => // TODO: Non-catalog paths for DSV2 are currently not well defined. userSpecifiedSchema match { - case Some(schema) => (provider.getTable(dsOptions, schema), null, null) - case _ => (provider.getTable(dsOptions), null, null) + case Some(schema) => (provider.getTable(dsOptions, schema), None, None) + case _ => (provider.getTable(dsOptions), None, None) } } import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index f18732a02186..c041d14c8b8d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -267,11 +267,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val catalog = CatalogV2Util.getTableProviderCatalog( supportsExtract, catalogManager, dsOptions) - (catalog.loadTable(ident), catalog, ident) + (catalog.loadTable(ident), Some(catalog), Some(ident)) case tableProvider: TableProvider => val t = tableProvider.getTable(dsOptions) if (t.supports(BATCH_WRITE)) { - (t, null, null) + (t, None, None) } else { // Streaming also uses the data source V2 API. So it may be that the data source // implements v2, but has no v2 implementation for batch writes. In that case, we @@ -418,7 +418,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case _: V1Table => return insertInto(TableIdentifier(ident.name(), ident.namespace().headOption)) case t => - DataSourceV2Relation.create(t, catalog, ident) + DataSourceV2Relation.create(t, Some(catalog), Some(ident)) } val command = mode match { @@ -558,7 +558,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case (SaveMode.Append, Some(table)) => checkPartitioningMatchesV2Table(table) - val v2Relation = DataSourceV2Relation.create(table, catalog, ident) + val v2Relation = DataSourceV2Relation.create(table, Some(catalog), Some(ident)) AppendData.byName(v2Relation, df.logicalPlan, extraOptions.toMap) case (SaveMode.Overwrite, _) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index 272c4dd1956d..7ade4d1bb46c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -160,7 +160,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) val append = loadTable(catalog, identifier) match { case Some(t) => AppendData.byName( - DataSourceV2Relation.create(t, catalog, identifier), + DataSourceV2Relation.create(t, Some(catalog), Some(identifier)), logicalPlan, options.toMap) case _ => throw new NoSuchTableException(identifier) @@ -184,7 +184,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) val overwrite = loadTable(catalog, identifier) match { case Some(t) => OverwriteByExpression.byName( - DataSourceV2Relation.create(t, catalog, identifier), + DataSourceV2Relation.create(t, Some(catalog), Some(identifier)), logicalPlan, condition.expr, options.toMap) case _ => throw new NoSuchTableException(identifier) @@ -211,7 +211,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) val dynamicOverwrite = loadTable(catalog, identifier) match { case Some(t) => OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(t, catalog, identifier), + DataSourceV2Relation.create(t, Some(catalog), Some(identifier)), logicalPlan, options.toMap) case _ => throw new NoSuchTableException(identifier) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index f3969a981e6c..9f544cb1cace 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -21,8 +21,6 @@ import java.util import scala.collection.JavaConverters._ -import org.scalatestplus.mockito.MockitoSugar - import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.analysis.{AnalysisSuite, NamedRelation} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} @@ -37,11 +35,9 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap -class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession with MockitoSugar { +class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { private val emptyMap = CaseInsensitiveStringMap.empty - private val mockCatalog = mock[CatalogPlugin] - private val mockIdent = mock[Identifier] private def createStreamingRelation(table: Table, v1Relation: Option[StreamingRelation]) = { StreamingRelationV2( TestTableProvider, @@ -60,8 +56,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession wi val e = intercept[AnalysisException] { TableCapabilityCheck.apply(DataSourceV2Relation.create( CapabilityTable(), - mockCatalog, - mockIdent, + None, + None, CaseInsensitiveStringMap.empty)) } assert(e.message.contains("does not support batch scan")) @@ -95,7 +91,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession wi test("AppendData: check missing capabilities") { val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(), mockCatalog, mockIdent, emptyMap), + DataSourceV2Relation.create(CapabilityTable(), None, mockIdent, emptyMap), TestRelation) val exc = intercept[AnalysisException]{ @@ -108,7 +104,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession wi test("AppendData: check correct capabilities") { Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(write), mockCatalog, mockIdent, emptyMap), + DataSourceV2Relation.create(CapabilityTable(write), None, mockIdent, emptyMap), TestRelation) TableCapabilityCheck.apply(plan) @@ -123,7 +119,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession wi CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), + DataSourceV2Relation.create(table, None, None, emptyMap), TestRelation, Literal(true)) @@ -142,7 +138,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession wi CapabilityTable(V1_BATCH_WRITE, OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), + DataSourceV2Relation.create(table, None, None, emptyMap), TestRelation, Literal(true)) @@ -157,7 +153,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession wi CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), + DataSourceV2Relation.create(table, None, None, emptyMap), TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) @@ -173,7 +169,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession wi Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val table = CapabilityTable(write, OVERWRITE_BY_FILTER) val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), + DataSourceV2Relation.create(table, None, None, emptyMap), TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) @@ -187,7 +183,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession wi CapabilityTable(OVERWRITE_DYNAMIC)).foreach { table => val plan = OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), + DataSourceV2Relation.create(table, None, None, emptyMap), TestRelation) val exc = intercept[AnalysisException] { @@ -201,7 +197,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession wi test("OverwritePartitionsDynamic: check correct capabilities") { val table = CapabilityTable(BATCH_WRITE, OVERWRITE_DYNAMIC) val plan = OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(table, mockCatalog, mockIdent, emptyMap), + DataSourceV2Relation.create(table, None, None, emptyMap), TestRelation) TableCapabilityCheck.apply(plan) From e59a38a9fa99080bb800e9ceb371e7865d3ad052 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 23 Jan 2020 22:34:55 -0800 Subject: [PATCH 21/23] fix the typo --- .../spark/sql/connector/SupportsCatalogOptionsSuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala index 00a7d491f93e..2a1f8c70eca8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala @@ -212,7 +212,6 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with val listener = new QueryExecutionListener { override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { plan = qe.analyzed - } override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {} } @@ -270,7 +269,7 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with private def save(name: String, mode: SaveMode, catalogOpt: Option[String]): Unit = { val df = spark.range(10).write.format(format).option("name", name) - catalogOpt.foreach(cName => df.option("catalog", catalogName)) + catalogOpt.foreach(cName => df.option("catalog", cName)) df.mode(mode).save() } } From 600d289dbf159c3a81d170016bf6f7831f9a3165 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Thu, 23 Jan 2020 22:45:17 -0800 Subject: [PATCH 22/23] fix tests --- .../sql/connector/catalog/CatalogV2UtilSuite.scala | 4 ++-- .../apache/spark/sql/DataFrameWriterV2Suite.scala | 4 ++-- .../sql/connector/SupportsCatalogOptionsSuite.scala | 4 ++-- .../sql/connector/TableCapabilityCheckSuite.scala | 4 ++-- .../sql/execution/command/PlanResolutionSuite.scala | 12 ++++++------ 5 files changed, 14 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala index bb1295768288..7a9a7f52ff8f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala @@ -34,7 +34,7 @@ class CatalogV2UtilSuite extends SparkFunSuite { assert(r.isDefined) assert(r.get.isInstanceOf[DataSourceV2Relation]) val v2Relation = r.get.asInstanceOf[DataSourceV2Relation] - assert(v2Relation.catalog == testCatalog) - assert(v2Relation.identifier == ident) + assert(v2Relation.catalog.exists(_ == testCatalog)) + assert(v2Relation.identifier.exists(_ == ident)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index ef347708e011..d49dc58e93dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -93,8 +93,8 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo catalogPlugin: TableCatalog = catalog("testcat")): Unit = { assert(plan.isInstanceOf[DataSourceV2Relation]) val v2 = plan.asInstanceOf[DataSourceV2Relation] - assert(v2.identifier.name() == identifier) - assert(v2.catalog == catalogPlugin) + assert(v2.identifier.exists(_.name() == identifier)) + assert(v2.catalog.exists(_ == catalogPlugin)) } test("Append: basic append") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala index 2a1f8c70eca8..cec48bb368ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala @@ -257,8 +257,8 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with catalogPlugin: TableCatalog = catalog(catalogName)): Unit = { assert(plan.isInstanceOf[DataSourceV2Relation]) val v2 = plan.asInstanceOf[DataSourceV2Relation] - assert(v2.identifier.name() == identifier) - assert(v2.catalog == catalogPlugin) + assert(v2.identifier.exists(_.name() == identifier)) + assert(v2.catalog.exists(_ == catalogPlugin)) } private def load(name: String, catalogOpt: Option[String]): DataFrame = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index 9f544cb1cace..236ff9d93e66 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -91,7 +91,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("AppendData: check missing capabilities") { val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(), None, mockIdent, emptyMap), + DataSourceV2Relation.create(CapabilityTable(), None, None, emptyMap), TestRelation) val exc = intercept[AnalysisException]{ @@ -104,7 +104,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("AppendData: check correct capabilities") { Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(write), None, mockIdent, emptyMap), + DataSourceV2Relation.create(CapabilityTable(write), None, None, emptyMap), TestRelation) TableCapabilityCheck.apply(plan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index c1a2f6164be3..70b9b7ec12ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -1103,14 +1103,14 @@ class PlanResolutionSuite extends AnalysisTest { val tableIdent = if (isSessionCatlog) "v2Table" else "tab" parsed match { case AlterTable(_, _, r: DataSourceV2Relation, _) => - assert(r.catalog == catlogIdent) - assert(r.identifier.name() == tableIdent) + assert(r.catalog.exists(_ == catlogIdent)) + assert(r.identifier.exists(_.name() == tableIdent)) case Project(_, r: DataSourceV2Relation) => - assert(r.catalog == catlogIdent) - assert(r.identifier.name() == tableIdent) + assert(r.catalog.exists(_ == catlogIdent)) + assert(r.identifier.exists(_.name() == tableIdent)) case InsertIntoStatement(r: DataSourceV2Relation, _, _, _, _) => - assert(r.catalog == catlogIdent) - assert(r.identifier.name() == tableIdent) + assert(r.catalog.exists(_ == catlogIdent)) + assert(r.identifier.exists(_.name() == tableIdent)) case DescribeRelation(r: ResolvedTable, _, _) => assert(r.catalog == catlogIdent) assert(r.identifier.name() == tableIdent) From df29683d38260cb7f69adfccdc7fab2d315e663e Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Fri, 24 Jan 2020 09:49:22 -0800 Subject: [PATCH 23/23] style --- .../spark/sql/connector/catalog/CatalogV2Util.scala | 4 +--- .../scala/org/apache/spark/sql/DataFrameWriterV2.scala | 1 - .../spark/sql/connector/TableCapabilityCheckSuite.scala | 8 +++----- 3 files changed, 4 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index bff1134621e4..7b2466bfe13e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -256,9 +256,7 @@ private[sql] object CatalogV2Util { case _: NoSuchNamespaceException => None } - def loadRelation( - catalog: CatalogPlugin, - ident: Identifier): Option[NamedRelation] = { + def loadRelation(catalog: CatalogPlugin, ident: Identifier): Option[NamedRelation] = { loadTable(catalog, ident).map(DataSourceV2Relation.create(_, Some(catalog), Some(ident))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index 7ade4d1bb46c..f5dd7613d410 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -53,7 +53,6 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) private val (catalog, identifier) = { val CatalogAndIdentifier(catalog, identifier) = tableName - (catalog.asTableCatalog, identifier) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index 236ff9d93e66..5196ca65276e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -54,11 +54,9 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("batch scan: check missing capabilities") { val e = intercept[AnalysisException] { - TableCapabilityCheck.apply(DataSourceV2Relation.create( - CapabilityTable(), - None, - None, - CaseInsensitiveStringMap.empty)) + TableCapabilityCheck.apply( + DataSourceV2Relation.create(CapabilityTable(), None, None, emptyMap) + ) } assert(e.message.contains("does not support batch scan")) }