Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._

import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, NamespaceChange, TableChange}
import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RemoveProperty, RenameColumn, SetProperty, UpdateColumnComment, UpdateColumnType}
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException}
import org.apache.spark.sql.sources.v2.Table
import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType}

Expand Down Expand Up @@ -219,5 +219,7 @@ object CatalogV2Util {
Option(catalog.asTableCatalog.loadTable(ident))
} catch {
case _: NoSuchTableException => None
case _: NoSuchDatabaseException => None
case _: NoSuchNamespaceException => None
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* 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.sources.v2.internal;

import org.apache.spark.sql.sources.v2.Table;

/**
* Internal interface used for table definitions, which we do not have complete information to
* resolve yet. This is primarily used by the `CatalogTableAsV2` wrapper returned by the
* V2SessionCatalog. When a `CatalogTableAsV2` is returned by the V2SessionCatalog, we defer
* planning to V1 data source code paths.
*/
public interface UnresolvedTable extends Table {}
Original file line number Diff line number Diff line change
Expand Up @@ -650,8 +650,11 @@ class Analyzer(
if catalog.isTemporaryTable(ident) =>
u // temporary views take precedence over catalog table names

case u @ UnresolvedRelation(CatalogObjectIdentifier(Some(catalogPlugin), ident)) =>
loadTable(catalogPlugin, ident).map(DataSourceV2Relation.create).getOrElse(u)
case u @ UnresolvedRelation(CatalogObjectIdentifier(maybeCatalog, ident)) =>
maybeCatalog.orElse(sessionCatalog)
.flatMap(loadTable(_, ident))
.map(DataSourceV2Relation.create)
.getOrElse(u)
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A +1 on this.

}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ 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.sources.v2._
import org.apache.spark.sql.sources.v2.internal.UnresolvedTable
import org.apache.spark.sql.sources.v2.reader.{Statistics => V2Statistics, _}
import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream}
import org.apache.spark.sql.sources.v2.writer._
Expand All @@ -42,6 +43,8 @@ case class DataSourceV2Relation(

import DataSourceV2Implicits._

override lazy val resolved: Boolean = !table.isInstanceOf[UnresolvedTable]

override def name: String = table.name()

override def skipSchemaResolution: Boolean = table.supports(TableCapability.ACCEPT_ANY_SCHEMA)
Expand Down Expand Up @@ -100,7 +103,10 @@ case class StreamingDataSourceV2Relation(

object DataSourceV2Relation {
def create(table: Table, options: CaseInsensitiveStringMap): DataSourceV2Relation = {
val output = table.schema().toAttributes
val output = table match {
case _: UnresolvedTable => Nil
case _ => table.schema().toAttributes
}
DataSourceV2Relation(table, output, options)
}

Expand Down
35 changes: 25 additions & 10 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode
import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister}
import org.apache.spark.sql.sources.v2._
import org.apache.spark.sql.sources.v2.TableCapability._
import org.apache.spark.sql.sources.v2.internal.UnresolvedTable
import org.apache.spark.sql.types.{IntegerType, StructType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap

Expand Down Expand Up @@ -251,19 +252,13 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
assertNotBucketed("save")

val session = df.sparkSession
val useV1Sources =
session.sessionState.conf.useV1SourceWriterList.toLowerCase(Locale.ROOT).split(",")
val cls = DataSource.lookupDataSource(source, session.sessionState.conf)
val shouldUseV1Source = cls.newInstance() match {
case d: DataSourceRegister if useV1Sources.contains(d.shortName()) => true
case _ => useV1Sources.contains(cls.getCanonicalName.toLowerCase(Locale.ROOT))
}
val canUseV2 = canUseV2Source(session, cls) && partitioningColumns.isEmpty

// In Data Source V2 project, partitioning is still under development.
// Here we fallback to V1 if partitioning columns are specified.
// TODO(SPARK-26778): use V2 implementations when partitioning feature is supported.
if (!shouldUseV1Source && classOf[TableProvider].isAssignableFrom(cls) &&
partitioningColumns.isEmpty) {
if (canUseV2) {
val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider]
val sessionOptions = DataSourceV2Utils.extractSessionConfigs(
provider, session.sessionState.conf)
Expand Down Expand Up @@ -493,13 +488,20 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier}
import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._

import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._
val session = df.sparkSession
val provider = DataSource.lookupDataSource(source, session.sessionState.conf)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the provider here may not be the actual table provider, as saveAsTable can write to an existing table. Maybe we should always use v2 session catalog?

Copy link
Contributor Author

@brkyvz brkyvz Aug 14, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That works for me. Since the V2 code path will fallback to the V1 code path if it sees an UnresolvedTable

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm. actually that causes issues if the table doesn't exist. Maybe we should use the statements instead of the logical plans?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 on using statements.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

let's do it in a followup.

val canUseV2 = canUseV2Source(session, provider)
val sessionCatalogOpt = session.sessionState.analyzer.sessionCatalog

session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match {
case CatalogObjectIdentifier(Some(catalog), ident) =>
saveAsTable(catalog.asTableCatalog, ident, modeForDSV2)
// TODO(SPARK-28666): This should go through V2SessionCatalog

case CatalogObjectIdentifier(None, ident)
if canUseV2 && sessionCatalogOpt.isDefined && ident.namespace().length <= 1 =>
// We pass in the modeForDSV1, as using the V2 session catalog should maintain compatibility
// for now.
saveAsTable(sessionCatalogOpt.get.asTableCatalog, ident, modeForDSV1)

case AsTableIdentifier(tableIdentifier) =>
saveAsTable(tableIdentifier)
Expand All @@ -525,6 +527,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
}

val command = (mode, tableOpt) match {
case (_, Some(table: UnresolvedTable)) =>
return saveAsTable(TableIdentifier(ident.name(), ident.namespace().headOption))

case (SaveMode.Append, Some(table)) =>
AppendData.byName(DataSourceV2Relation.create(table), df.logicalPlan)

Expand Down Expand Up @@ -830,6 +835,16 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {

private def modeForDSV2 = mode.getOrElse(SaveMode.Append)

private def canUseV2Source(session: SparkSession, providerClass: Class[_]): Boolean = {
val useV1Sources =
session.sessionState.conf.useV1SourceWriterList.toLowerCase(Locale.ROOT).split(",")
val shouldUseV1Source = providerClass.newInstance() match {
case d: DataSourceRegister if useV1Sources.contains(d.shortName()) => true
case _ => useV1Sources.contains(providerClass.getCanonicalName.toLowerCase(Locale.ROOT))
}
!shouldUseV1Source && classOf[TableProvider].isAssignableFrom(providerClass)
}

///////////////////////////////////////////////////////////////////////////////////////
// Builder pattern config options
///////////////////////////////////////////////////////////////////////////////////////
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,17 +24,17 @@ import scala.collection.mutable
import org.apache.spark.sql.{AnalysisException, SaveMode}
import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, LookupCatalog, TableCatalog}
import org.apache.spark.sql.catalog.v2.expressions.Transform
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

dammit IntelliJ :(

import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, ReplaceTable, ReplaceTableAsSelect}
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.logical.sql._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand}
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.datasources.v2.{CatalogTableAsV2, DataSourceV2Relation}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.v2.TableProvider
import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType}
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.SchemaUtils

case class DataSourceResolution(
Expand Down Expand Up @@ -173,8 +173,10 @@ case class DataSourceResolution(
// only top-level adds are supported using AlterTableAddColumnsCommand
AlterTableAddColumnsCommand(table, newColumns.map(convertToStructField))

case DataSourceV2Relation(CatalogTableAsV2(catalogTable), _, _) =>
UnresolvedCatalogRelation(catalogTable)
case DataSourceV2Relation(CatalogTableAsV2(ct), _, _) =>
SubqueryAlias(
AliasIdentifier(ct.identifier.table, ct.identifier.database),
UnresolvedCatalogRelation(ct))

}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogT
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.internal.SessionState
import org.apache.spark.sql.sources.v2.{Table, TableCapability}
import org.apache.spark.sql.sources.v2.internal.UnresolvedTable
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap

Expand Down Expand Up @@ -172,7 +173,7 @@ class V2SessionCatalog(sessionState: SessionState) extends TableCatalog {
/**
* An implementation of catalog v2 [[Table]] to expose v1 table metadata.
*/
case class CatalogTableAsV2(v1Table: CatalogTable) extends Table {
case class CatalogTableAsV2(v1Table: CatalogTable) extends UnresolvedTable {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we move CatalogTableAsV2 to catalyst and rename it to UnresolvedTable, then we don't need to create an extra interface. what do you think? @brkyvz @rdblue

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

CatalogTable is defined in sql unfortunately.

Copy link
Contributor

@cloud-fan cloud-fan Aug 14, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's defined in catalyst: org.apache.spark.sql.catalyst.catalog.CatalogTable in file sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, I like that a lot more

implicit class IdentifierHelper(identifier: TableIdentifier) {
def quoted: String = {
identifier.database match {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
/*
* 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.sources.v2

import java.util
import java.util.concurrent.ConcurrentHashMap

import scala.collection.JavaConverters._

import org.scalatest.BeforeAndAfter

import org.apache.spark.sql.{DataFrame, QueryTest}
import org.apache.spark.sql.catalog.v2.Identifier
import org.apache.spark.sql.catalog.v2.expressions.Transform
import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap

class DataSourceV2DataFrameSessionCatalogSuite
extends QueryTest
with SharedSQLContext
with BeforeAndAfter {
import testImplicits._

private val v2Format = classOf[InMemoryTableProvider].getName

before {
spark.conf.set(SQLConf.V2_SESSION_CATALOG.key, classOf[TestV2SessionCatalog].getName)
}

override def afterEach(): Unit = {
super.afterEach()
spark.catalog("session").asInstanceOf[TestV2SessionCatalog].clearTables()
}

private def verifyTable(tableName: String, expected: DataFrame): Unit = {
checkAnswer(spark.table(tableName), expected)
checkAnswer(sql(s"SELECT * FROM $tableName"), expected)
checkAnswer(sql(s"TABLE $tableName"), expected)
}

test("saveAsTable and v2 table - table doesn't exist") {
val t1 = "tbl"
val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
df.write.format(v2Format).saveAsTable(t1)
verifyTable(t1, df)
}

test("saveAsTable: v2 table - table exists") {
val t1 = "tbl"
val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
spark.sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format")
intercept[TableAlreadyExistsException] {
df.select("id", "data").write.format(v2Format).saveAsTable(t1)
}
df.write.format(v2Format).mode("append").saveAsTable(t1)
verifyTable(t1, df)

// Check that appends are by name
df.select('data, 'id).write.format(v2Format).mode("append").saveAsTable(t1)
Copy link
Contributor

@cloud-fan cloud-fan Aug 13, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IIRC, in DS v1, saveAsTable fails if the table exists, but the table provider is different from the one specified in df.write.format. Do we have this check in the v2 code path?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll add a test

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since the provider isn't necessarily exposed by the table API, I'm not sure if such a check is required/possible.

verifyTable(t1, df.union(df))
}

test("saveAsTable: v2 table - table overwrite and table doesn't exist") {
val t1 = "tbl"
val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
df.write.format(v2Format).mode("overwrite").saveAsTable(t1)
verifyTable(t1, df)
}

test("saveAsTable: v2 table - table overwrite and table exists") {
val t1 = "tbl"
val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
spark.sql(s"CREATE TABLE $t1 USING $v2Format AS SELECT 'c', 'd'")
df.write.format(v2Format).mode("overwrite").saveAsTable(t1)
verifyTable(t1, df)
}

test("saveAsTable: v2 table - ignore mode and table doesn't exist") {
val t1 = "tbl"
val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
df.write.format(v2Format).mode("ignore").saveAsTable(t1)
verifyTable(t1, df)
}

test("saveAsTable: v2 table - ignore mode and table exists") {
val t1 = "tbl"
val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
spark.sql(s"CREATE TABLE $t1 USING $v2Format AS SELECT 'c', 'd'")
df.write.format(v2Format).mode("ignore").saveAsTable(t1)
verifyTable(t1, Seq(("c", "d")).toDF("id", "data"))
}
}

class InMemoryTableProvider extends TableProvider {
override def getTable(options: CaseInsensitiveStringMap): Table = {
throw new UnsupportedOperationException("D'oh!")
}
}

/** A SessionCatalog that always loads an in memory Table, so we can test write code paths. */
class TestV2SessionCatalog extends V2SessionCatalog {

protected val tables: util.Map[Identifier, InMemoryTable] =
new ConcurrentHashMap[Identifier, InMemoryTable]()

override def loadTable(ident: Identifier): Table = {
if (tables.containsKey(ident)) {
tables.get(ident)
} else {
// Table was created through the built-in catalog
val t = super.loadTable(ident)
val table = new InMemoryTable(t.name(), t.schema(), t.partitioning(), t.properties())
tables.put(ident, table)
table
}
}

override def createTable(
ident: Identifier,
schema: StructType,
partitions: Array[Transform],
properties: util.Map[String, String]): Table = {
val t = new InMemoryTable(ident.name(), schema, partitions, properties)
tables.put(ident, t)
t
}

def clearTables(): Unit = {
assert(!tables.isEmpty, "Tables were empty, maybe didn't use the session catalog code path?")
tables.keySet().asScala.foreach(super.dropTable)
tables.clear()
}
}