Skip to content

Commit

Permalink
[SPARK-29219][SQL] Introduce SupportsCatalogOptions for TableProvider
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

This PR introduces `SupportsCatalogOptions` as an interface for `TableProvider`. Through `SupportsCatalogOptions`, V2 DataSources can implement the two methods `extractIdentifier` and `extractCatalog` to support the creation, and existence check of tables without requiring a formal TableCatalog implementation.

We currently don't support all SaveModes for DataSourceV2 in DataFrameWriter.save. The idea here is that eventually File based tables can be written with `DataFrameWriter.save(path)` will create a PathIdentifier where the name is `path`, and the V2SessionCatalog will be able to perform FileSystem checks at `path` to support ErrorIfExists and Ignore SaveModes.

### Why are the changes needed?

To support all Save modes for V2 data sources with DataFrameWriter. Since we can now support table creation, we will be able to provide partitioning information when first creating the table as well.

### Does this PR introduce any user-facing change?

Introduces a new interface

### How was this patch tested?

Will add tests once interface is vetted.

Closes #26913 from brkyvz/catalogOptions.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
  • Loading branch information
brkyvz and brkyvz committed Jan 9, 2020
1 parent c88124a commit f8d5957
Show file tree
Hide file tree
Showing 7 changed files with 406 additions and 44 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets.UTF_8
import java.util.concurrent.atomic.AtomicInteger

import scala.reflect.ClassTag
import scala.util.Try

import org.apache.kafka.clients.producer.ProducerConfig
import org.apache.kafka.clients.producer.internals.DefaultPartitioner
Expand Down Expand Up @@ -500,7 +501,7 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase {
TestUtils.assertExceptionMsg(ex, "null topic present in the data")
}

protected def testUnsupportedSaveModes(msg: (SaveMode) => String): Unit = {
protected def testUnsupportedSaveModes(msg: (SaveMode) => Seq[String]): Unit = {
val topic = newTopic()
testUtils.createTopic(topic)
val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value")
Expand All @@ -513,7 +514,10 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase {
.mode(mode)
.save()
}
TestUtils.assertExceptionMsg(ex, msg(mode))
val errorChecks = msg(mode).map(m => Try(TestUtils.assertExceptionMsg(ex, m)))
if (!errorChecks.exists(_.isSuccess)) {
fail("Error messages not found in exception trace")
}
}
}

Expand Down Expand Up @@ -541,7 +545,7 @@ class KafkaSinkBatchSuiteV1 extends KafkaSinkBatchSuiteBase {
.set(SQLConf.USE_V1_SOURCE_LIST, "kafka")

test("batch - unsupported save modes") {
testUnsupportedSaveModes((mode) => s"Save mode ${mode.name} not allowed for Kafka")
testUnsupportedSaveModes((mode) => s"Save mode ${mode.name} not allowed for Kafka" :: Nil)
}
}

Expand All @@ -552,7 +556,8 @@ class KafkaSinkBatchSuiteV2 extends KafkaSinkBatchSuiteBase {
.set(SQLConf.USE_V1_SOURCE_LIST, "")

test("batch - unsupported save modes") {
testUnsupportedSaveModes((mode) => s"cannot be written with ${mode.name} mode")
testUnsupportedSaveModes((mode) =>
Seq(s"cannot be written with ${mode.name} mode", "does not support truncate"))
}

test("generic - write big data with small producer buffer") {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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.apache.spark.annotation.Evolving;
import org.apache.spark.sql.util.CaseInsensitiveStringMap;

/**
* An interface, which TableProviders can implement, to support table existence checks and creation
* through a catalog, without having to use table identifiers. For example, when file based data
* sources use the `DataFrameWriter.save(path)` method, the option `path` can translate to a
* PathIdentifier. A catalog can then use this PathIdentifier to check the existence of a table, or
* whether a table can be created at a given directory.
*/
@Evolving
public interface SupportsCatalogOptions extends TableProvider {
/**
* Return a {@link Identifier} instance that can identify a table for a DataSource given
* DataFrame[Reader|Writer] options.
*
* @param options the user-specified options that can identify a table, e.g. file path, Kafka
* topic name, etc. It's an immutable case-insensitive string-to-string map.
*/
Identifier extractIdentifier(CaseInsensitiveStringMap options);

/**
* Return the name of a catalog that can be used to check the existence of, load, and create
* a table for this DataSource given the identifier that will be extracted by
* {@link #extractIdentifier(CaseInsensitiveStringMap) extractIdentifier}. A `null` value can
* be used to defer to the V2SessionCatalog.
*
* @param options the user-specified options that can identify a table, e.g. file path, Kafka
* topic name, etc. It's an immutable case-insensitive string-to-string map.
*/
default String extractCatalog(CaseInsensitiveStringMap options) {
return CatalogManager.SESSION_CATALOG_NAME();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical.AlterTable
import org.apache.spark.sql.connector.catalog.TableChange._
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap

private[sql] object CatalogV2Util {
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
Expand Down Expand Up @@ -315,4 +316,14 @@ private[sql] object CatalogV2Util {
val unresolved = UnresolvedV2Relation(originalNameParts, tableCatalog, ident)
AlterTable(tableCatalog, ident, unresolved, changes)
}

def getTableProviderCatalog(
provider: SupportsCatalogOptions,
catalogManager: CatalogManager,
options: CaseInsensitiveStringMap): TableCatalog = {
Option(provider.extractCatalog(options))
.map(catalogManager.catalog)
.getOrElse(catalogManager.v2SessionCatalog)
.asTableCatalog
}
}
21 changes: 17 additions & 4 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, Univocit
import org.apache.spark.sql.catalyst.expressions.ExprUtils
import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions}
import org.apache.spark.sql.catalyst.util.FailureSafeParser
import org.apache.spark.sql.connector.catalog.SupportsRead
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsCatalogOptions, SupportsRead}
import org.apache.spark.sql.connector.catalog.TableCapability._
import org.apache.spark.sql.execution.command.DDLUtils
import org.apache.spark.sql.execution.datasources.DataSource
Expand Down Expand Up @@ -206,9 +206,22 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {

val finalOptions = sessionOptions ++ extraOptions.toMap ++ pathsOption
val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava)
val table = userSpecifiedSchema match {
case Some(schema) => provider.getTable(dsOptions, schema)
case _ => provider.getTable(dsOptions)
val table = provider match {
case _: SupportsCatalogOptions if userSpecifiedSchema.nonEmpty =>
throw new IllegalArgumentException(
s"$source does not support user specified schema. Please don't specify the schema.")
case hasCatalog: SupportsCatalogOptions =>
val ident = hasCatalog.extractIdentifier(dsOptions)
val catalog = CatalogV2Util.getTableProviderCatalog(
hasCatalog,
sparkSession.sessionState.catalogManager,
dsOptions)
catalog.loadTable(ident)
case _ =>
userSpecifiedSchema match {
case Some(schema) => provider.getTable(dsOptions, schema)
case _ => provider.getTable(dsOptions)
}
}
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._
table match {
Expand Down
128 changes: 92 additions & 36 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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.{CatalogPlugin, CatalogV2Implicits, CatalogV2Util, Identifier, SupportsCatalogOptions, SupportsWrite, Table, 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
Expand Down Expand Up @@ -258,37 +258,77 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
val dsOptions = new CaseInsensitiveStringMap(options.asJava)

import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._
provider.getTable(dsOptions) match {
case table: SupportsWrite if table.supports(BATCH_WRITE) =>
if (partitioningColumns.nonEmpty) {
throw new AnalysisException("Cannot write data to TableProvider implementation " +
"if partition columns are specified.")
}
lazy val relation = DataSourceV2Relation.create(table, dsOptions)
mode match {
case SaveMode.Append =>
runCommand(df.sparkSession, "save") {
AppendData.byName(relation, df.logicalPlan, extraOptions.toMap)
mode match {
case SaveMode.Append | SaveMode.Overwrite =>
val table = provider match {
case supportsExtract: SupportsCatalogOptions =>
val ident = supportsExtract.extractIdentifier(dsOptions)
val sessionState = df.sparkSession.sessionState
val catalog = CatalogV2Util.getTableProviderCatalog(
supportsExtract, sessionState.catalogManager, dsOptions)

catalog.loadTable(ident)
case tableProvider: TableProvider =>
val t = tableProvider.getTable(dsOptions)
if (t.supports(BATCH_WRITE)) {
t
} 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
// fall back to saving as though it's a V1 source.
return saveToV1Source()
}
}

val relation = DataSourceV2Relation.create(table, dsOptions)
checkPartitioningMatchesV2Table(table)
if (mode == SaveMode.Append) {
runCommand(df.sparkSession, "save") {
AppendData.byName(relation, df.logicalPlan, extraOptions.toMap)
}
} else {
// Truncate the table. TableCapabilityCheck will throw a nice exception if this
// isn't supported
runCommand(df.sparkSession, "save") {
OverwriteByExpression.byName(
relation, df.logicalPlan, Literal(true), extraOptions.toMap)
}
}

case createMode =>
provider match {
case supportsExtract: SupportsCatalogOptions =>
val ident = supportsExtract.extractIdentifier(dsOptions)
val sessionState = df.sparkSession.sessionState
val catalog = CatalogV2Util.getTableProviderCatalog(
supportsExtract, sessionState.catalogManager, dsOptions)

val location = Option(dsOptions.get("path")).map(TableCatalog.PROP_LOCATION -> _)

case SaveMode.Overwrite if table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER) =>
// truncate the table
runCommand(df.sparkSession, "save") {
OverwriteByExpression.byName(
relation, df.logicalPlan, Literal(true), extraOptions.toMap)
CreateTableAsSelect(
catalog,
ident,
partitioningAsV2,
df.queryExecution.analyzed,
Map(TableCatalog.PROP_PROVIDER -> source) ++ location,
extraOptions.toMap,
ignoreIfExists = createMode == SaveMode.Ignore)
}
case tableProvider: TableProvider =>
if (tableProvider.getTable(dsOptions).supports(BATCH_WRITE)) {
throw new AnalysisException(s"TableProvider implementation $source cannot be " +
s"written with $createMode mode, please use Append or Overwrite " +
"modes instead.")
} 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
// fallback to saving as though it's a V1 source.
saveToV1Source()
}

case other =>
throw new AnalysisException(s"TableProvider implementation $source cannot be " +
s"written with $other mode, please use Append or Overwrite " +
"modes instead.")
}

// 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 fall back to saving
// as though it's a V1 source.
case _ => saveToV1Source()
}

} else {
saveToV1Source()
}
Expand Down Expand Up @@ -504,14 +544,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {


private def saveAsTable(catalog: TableCatalog, ident: Identifier): Unit = {
val partitioning = partitioningColumns.map { colNames =>
colNames.map(name => IdentityTransform(FieldReference(name)))
}.getOrElse(Seq.empty[Transform])
val bucketing = bucketColumnNames.map { cols =>
Seq(BucketTransform(LiteralValue(numBuckets.get, IntegerType), cols.map(FieldReference(_))))
}.getOrElse(Seq.empty[Transform])
val partitionTransforms = partitioning ++ bucketing

val tableOpt = try Option(catalog.loadTable(ident)) catch {
case _: NoSuchTableException => None
}
Expand All @@ -526,13 +558,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
return saveAsTable(TableIdentifier(ident.name(), ident.namespace().headOption))

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

case (SaveMode.Overwrite, _) =>
ReplaceTableAsSelect(
catalog,
ident,
partitionTransforms,
partitioningAsV2,
df.queryExecution.analyzed,
Map(TableCatalog.PROP_PROVIDER -> source) ++ getLocationIfExists,
extraOptions.toMap,
Expand All @@ -545,7 +578,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
CreateTableAsSelect(
catalog,
ident,
partitionTransforms,
partitioningAsV2,
df.queryExecution.analyzed,
Map(TableCatalog.PROP_PROVIDER -> source) ++ getLocationIfExists,
extraOptions.toMap,
Expand Down Expand Up @@ -623,6 +656,29 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
CreateTable(tableDesc, mode, Some(df.logicalPlan)))
}

/** Converts the provided partitioning and bucketing information to DataSourceV2 Transforms. */
private def partitioningAsV2: Seq[Transform] = {
val partitioning = partitioningColumns.map { colNames =>
colNames.map(name => IdentityTransform(FieldReference(name)))
}.getOrElse(Seq.empty[Transform])
val bucketing =
getBucketSpec.map(spec => CatalogV2Implicits.BucketSpecHelper(spec).asTransform).toSeq
partitioning ++ bucketing
}

/**
* For V2 DataSources, performs if the provided partitioning matches that of the table.
* Partitioning information is not required when appending data to V2 tables.
*/
private def checkPartitioningMatchesV2Table(existingTable: Table): Unit = {
val v2Partitions = partitioningAsV2
if (v2Partitions.isEmpty) return
require(v2Partitions.sameElements(existingTable.partitioning()),
"The provided partitioning does not match of the table.\n" +
s" - provided: ${v2Partitions.mkString(", ")}\n" +
s" - table: ${existingTable.partitioning().mkString(", ")}")
}

/**
* Saves the content of the `DataFrame` to an external database table via JDBC. In the case the
* table already exists in the external database, behavior of this function depends on the
Expand Down
Loading

0 comments on commit f8d5957

Please sign in to comment.