Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-29219][SQL] Introduce SupportsCatalogOptions for TableProvider #26913

Closed
wants to merge 14 commits into from
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 null;
Copy link
Contributor

Choose a reason for hiding this comment

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

shall we by default return CatalogManager.SESSION_CATALOG_NAME instead of null?

}
}
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
}
}
18 changes: 14 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 @@ -215,9 +215,19 @@ 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 hasCatalog: SupportsCatalogOptions =>
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 fail if the user specifies schema.

val ident = hasCatalog.extractIdentifier(dsOptions)
val catalog = CatalogV2Util.getTableProviderCatalog(
hasCatalog,
sparkSession.sessionState.catalogManager,
dsOptions)
catalog.loadTable(ident)
Copy link
Contributor

Choose a reason for hiding this comment

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

shall we always call TableProvider.getTable?

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 call loadTable, how do we handle user-specified schema?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We don't, we simply ignore it. If the TableProvider SupportsCatalogOptions, then we will always load the table through the catalog, therefore we don't need user options or partitioning info

Copy link
Contributor

Choose a reason for hiding this comment

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

then shall we fail or log a warning if schema is specified by users?

BTW SupportsCatalogOptions seems not a mixin as it doesn't need anything from TableProvider.

Copy link
Contributor

Choose a reason for hiding this comment

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

I don't have a strong opinion on whether to warn, fail, or ignore when there is a user-specified schema. Warnings are almost always ignored, so I'd rather fail and highlight the problem to the user.

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
65 changes: 50 additions & 15 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, 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 @@ -260,24 +260,42 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
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 =>
checkPartitioningMatchesV2Table(table)
runCommand(df.sparkSession, "save") {
AppendData.byName(relation, df.logicalPlan, extraOptions.toMap)
}

case SaveMode.Overwrite if table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER) =>
checkPartitioningMatchesV2Table(table)
// truncate the table
runCommand(df.sparkSession, "save") {
OverwriteByExpression.byName(
relation, df.logicalPlan, Literal(true), extraOptions.toMap)
}

case other if classOf[SupportsCatalogOptions].isAssignableFrom(provider.getClass) =>
val catalogOptions = provider.asInstanceOf[SupportsCatalogOptions]
val ident = catalogOptions.extractIdentifier(dsOptions)
val sessionState = df.sparkSession.sessionState
val catalog = CatalogV2Util.getTableProviderCatalog(
catalogOptions, sessionState.catalogManager, dsOptions)

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

runCommand(df.sparkSession, "save") {
CreateTableAsSelect(
catalog,
ident,
partitioningAsV2,
df.queryExecution.analyzed,
Map(TableCatalog.PROP_PROVIDER -> source) ++ location,
extraOptions.toMap,
ignoreIfExists = other == SaveMode.Ignore)
}

case other =>
throw new AnalysisException(s"TableProvider implementation $source cannot be " +
s"written with $other mode, please use Append or Overwrite " +
Expand Down Expand Up @@ -504,14 +522,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 +536,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 +556,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 +634,30 @@ 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 = bucketColumnNames.map { cols =>
Copy link
Contributor

Choose a reason for hiding this comment

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

shall we call CatalogV2Implicits.BucketSpecHelper.asTransform?

Seq(BucketTransform(LiteralValue(numBuckets.get, IntegerType), cols.map(FieldReference(_))))
}.getOrElse(Seq.empty[Transform])
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
Original file line number Diff line number Diff line change
@@ -0,0 +1,145 @@
/*
* 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

import scala.language.implicitConversions

import org.scalatest.BeforeAndAfter

import org.apache.spark.sql.{QueryTest, SaveMode}
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.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

class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with BeforeAndAfter {

import testImplicits._

private val catalogName = "testcat"
private val format = classOf[CatalogSupportingInMemoryTableProvider].getName

private def catalog(name: String): InMemoryTableSessionCatalog = {
spark.sessionState.catalogManager.catalog(name).asInstanceOf[InMemoryTableSessionCatalog]
}

private implicit def stringToIdentifier(value: String): Identifier = {
Identifier.of(Array.empty, value)
}

before {
spark.conf.set(
V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[InMemoryTableSessionCatalog].getName)
spark.conf.set(
s"spark.sql.catalog.$catalogName", classOf[InMemoryTableSessionCatalog].getName)
}

override def afterEach(): Unit = {
super.afterEach()
catalog(SESSION_CATALOG_NAME).clearTables()
catalog(catalogName).clearTables()
spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key)
spark.conf.unset(s"spark.sql.catalog.$catalogName")
}

def testWithDifferentCatalogs(withCatalogOption: Option[String]): Unit = {
Seq(SaveMode.ErrorIfExists, SaveMode.Ignore).foreach { saveMode =>
test(s"save works with $saveMode - no table, no partitioning, session catalog, " +
s"withCatalog: ${withCatalogOption.isDefined}") {
val df = spark.range(10)
val dfw = df.write.format(format).mode(saveMode).option("name", "t1")
withCatalogOption.foreach(cName => dfw.option("catalog", cName))
dfw.save()

val table = catalog(withCatalogOption.getOrElse(SESSION_CATALOG_NAME)).loadTable("t1")
assert(table.name() === "t1", "Table identifier was wrong")
assert(table.partitioning().isEmpty, "Partitioning should be empty")
assert(table.schema() === df.schema.asNullable, "Schema did not match")

val dfr = spark.read.format(format).option("name", "t1")
withCatalogOption.foreach(cName => dfr.option("catalog", cName))
checkAnswer(dfr.load(), df.toDF())
}

test(s"save works with $saveMode - no table, with partitioning, session catalog, " +
s"withCatalog: ${withCatalogOption.isDefined}") {
val df = spark.range(10).withColumn("part", 'id % 5)
val dfw = df.write.format(format).mode(saveMode).option("name", "t1").partitionBy("part")
withCatalogOption.foreach(cName => dfw.option("catalog", cName))
dfw.save()

val table = catalog(withCatalogOption.getOrElse(SESSION_CATALOG_NAME)).loadTable("t1")
assert(table.name() === "t1", "Table identifier was wrong")
assert(table.partitioning().length === 1, "Partitioning should not be empty")
assert(table.partitioning().head.references().head.fieldNames().head === "part",
"Partitioning was incorrect")
assert(table.schema() === df.schema.asNullable, "Schema did not match")

val dfr = spark.read.format(format).option("name", "t1")
withCatalogOption.foreach(cName => dfr.option("catalog", cName))
checkAnswer(dfr.load(), df.toDF())
}
}

test(s"save fails with ErrorIfExists if table exists, withCatalog: ${withCatalogOption.isDefined}") {
sql("create table t1 (id bigint) using foo")
val df = spark.range(10)
intercept[TableAlreadyExistsException] {
val dfw = df.write.format(format).option("name", "t1")
withCatalogOption.foreach(cName => dfw.option("catalog", cName))
dfw.save()
}
}

test(s"Ignore mode if table exists, withCatalog: ${withCatalogOption.isDefined}") {
sql("create table t1 (id bigint) using foo")
val df = spark.range(10).withColumn("part", 'id % 5)
intercept[TableAlreadyExistsException] {
val dfw = df.write.format(format).mode(SaveMode.Ignore).option("name", "t1")
withCatalogOption.foreach(cName => dfw.option("catalog", cName))
dfw.save()
}

val table = catalog(SESSION_CATALOG_NAME).loadTable("t1")
assert(table.partitioning().isEmpty, "Partitioning should be empty")
assert(table.schema() === new StructType().add("id", LongType), "Schema did not match")
}
}

testWithDifferentCatalogs(None)

testWithDifferentCatalogs(Some(catalogName))
}

class CatalogSupportingInMemoryTableProvider
extends InMemoryTableProvider
with SupportsCatalogOptions {

override def extractIdentifier(options: CaseInsensitiveStringMap): Identifier = {
val name = options.get("name")
assert(name != null, "The name should be provided for this table")
Identifier.of(Array.empty, name)
}

override def extractCatalog(options: CaseInsensitiveStringMap): String = {
options.get("catalog")
}
}