Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -34,10 +34,11 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, Project}
import org.apache.spark.sql.catalyst.plans.logical.{AppendData, InsertIntoDir, InsertIntoTable, LogicalPlan, OverwriteByExpression, Project}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan}
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -240,6 +241,12 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan]
})
}

private def fallbackToV1Relation(
t: DataSourceV1Table,
existingOutput: Seq[AttributeReference]): LogicalRelation = {
LogicalRelation(t.v1Relation, existingOutput, catalogTable = None, isStreaming = false)
}

override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case i @ InsertIntoTable(UnresolvedCatalogRelation(tableMeta), _, _, _, _)
if DDLUtils.isDatasourceTable(tableMeta) =>
Expand All @@ -253,6 +260,16 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan]

case UnresolvedCatalogRelation(tableMeta) =>
DDLUtils.readHiveTable(tableMeta)

case AppendData(DataSourceV2Relation(t: DataSourceV1Table, output, _), query, false) =>
InsertIntoDataSourceCommand(fallbackToV1Relation(t, output), query, overwrite = false)

case OverwriteByExpression(
DataSourceV2Relation(t: DataSourceV1Table, output, _), Literal(true, _), query, false) =>
Copy link
Contributor Author

Choose a reason for hiding this comment

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

The Data Source V1 API don't support static partition overwrite (e.g. INSERT OVERWRITE t PARTITION(a=1) SELECT ...). We must fully migrate JDBC to Data Source V2 before supporting it.

Here we only need to match the OverwriteByExpression(..., deleteExpr=Literal(true)), as this is the only thing DS v1 supports.

Copy link
Contributor

Choose a reason for hiding this comment

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

maybe also add a @TODO here?

InsertIntoDataSourceCommand(fallbackToV1Relation(t, output), query, overwrite = true)

case DataSourceV2Relation(t: DataSourceV1Table, output, _) =>
fallbackToV1Relation(t, output)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -183,7 +183,7 @@ object JdbcUtils extends Logging {
}
}

private def getJdbcType(dt: DataType, dialect: JdbcDialect): JdbcType = {
def getJdbcType(dt: DataType, dialect: JdbcDialect): JdbcType = {
dialect.getJDBCType(dt).orElse(getCommonJDBCType(dt)).getOrElse(
throw new IllegalArgumentException(s"Can't get JDBC type for ${dt.catalogString}"))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec}
import org.apache.spark.sql.sources
import org.apache.spark.sql.sources.DataSourceV1TableCatalog
import org.apache.spark.sql.sources.v2.reader._
import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
Expand Down Expand Up @@ -170,6 +171,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
case staging: StagingTableCatalog =>
AtomicCreateTableAsSelectExec(
staging, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil
case v1Catalog: DataSourceV1TableCatalog =>
CreateV1TableAsSelectExec(
v1Catalog, ident, parts, query, props, writeOptions, ifNotExists) :: Nil
case _ =>
CreateTableAsSelectExec(
catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,14 +26,15 @@ import org.apache.spark.{SparkEnv, SparkException, TaskContext}
import org.apache.spark.executor.CommitDeniedException
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.Dataset
import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog}
import org.apache.spark.sql.catalog.v2.expressions.Transform
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException}
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
import org.apache.spark.sql.sources.{AlwaysTrue, Filter}
import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, UnaryExecNode}
import org.apache.spark.sql.sources.{AlwaysTrue, DataSourceV1Table, DataSourceV1TableCatalog, Filter, InsertableRelation}
import org.apache.spark.sql.sources.v2.{StagedTable, SupportsWrite}
import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
Expand All @@ -50,6 +51,43 @@ case class WriteToDataSourceV2(batchWrite: BatchWrite, query: LogicalPlan)
override def output: Seq[Attribute] = Nil
}

case class CreateV1TableAsSelectExec(
catalog: DataSourceV1TableCatalog,
ident: Identifier,
partitioning: Seq[Transform],
query: LogicalPlan,
properties: Map[String, String],
writeOptions: CaseInsensitiveStringMap,
ifNotExists: Boolean) extends LeafExecNode {

override def output: Seq[Attribute] = Nil

override protected def doExecute(): RDD[InternalRow] = {
if (catalog.tableExists(ident)) {
if (ifNotExists) {
return sparkContext.emptyRDD[InternalRow]
}

throw new TableAlreadyExistsException(ident)
}

Utils.tryWithSafeFinallyAndFailureCallbacks({
catalog.createTable(
ident, query.schema, partitioning.toArray, properties.asJava) match {
case table: DataSourceV1Table =>
table.v1Relation.asInstanceOf[InsertableRelation].insert(
Dataset.ofRows(sqlContext.sparkSession, query), overwrite = false)
sparkContext.emptyRDD[InternalRow]

case _ =>
throw new SparkException(s"DataSourceV1TableCatalog must create DataSourceV1Table.")
}
})(catchBlock = {
catalog.dropTable(ident)
})
}
}

/**
* Physical plan node for v2 create table as select when the catalog does not support staging
* the table creation.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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.execution.datasources.v2.jdbc

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalog.v2.Identifier
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCPartition, JDBCRelation}
import org.apache.spark.sql.sources.{BaseRelation, DataSourceV1Table}
import org.apache.spark.sql.types.StructType

case class JDBCTable(
ident: Identifier,
schema: StructType,
jdbcOptions: JDBCOptions) extends DataSourceV1Table {
assert(ident.namespace().length == 1)

override def name(): String = ident.toString

override def v1Relation: BaseRelation = {
JDBCRelation(
schema,
// TODO: support column partitioning after we support table properties in JDBC table.
Array(JDBCPartition(null, 0)),
jdbcOptions)(SparkSession.active)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
/*
* 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.execution.datasources.v2.jdbc

import java.sql.{Connection, SQLException}
import java.util

import scala.collection.JavaConverters._

import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalog.v2.{Identifier, TableChange}
import org.apache.spark.sql.catalog.v2.expressions.Transform
import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException}
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCRDD, JdbcUtils}
import org.apache.spark.sql.jdbc.JdbcDialects
import org.apache.spark.sql.sources.DataSourceV1TableCatalog
import org.apache.spark.sql.sources.v2.Table
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap

class JDBCTableCatalog extends DataSourceV1TableCatalog with Logging {

private var _name: String = _
private var options: JDBCOptions = _

override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = {
_name = name
val map = options.asCaseSensitiveMap().asScala.toMap
// The `JDBCOptions` checks the existence of the table option. This is required by JDBC v1, but
// JDBC V2 only knows the table option when loading a table. Here we put a table option with a
Copy link
Contributor

Choose a reason for hiding this comment

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

same upper case V1 and V2?

// fake value, so that it can pass the check of `JDBCOptions`.
this.options = new JDBCOptions(map + (JDBCOptions.JDBC_TABLE_NAME -> "__invalid"))
}

override def name(): String = {
_name
}

private def withConnection[T](f: Connection => T): T = {
val conn = JdbcUtils.createConnectionFactory(options)()
try {
f(conn)
} finally {
conn.close()
}
}

private def checkNamespace(namespace: Array[String]): Unit = {
// In JDBC the tables must be in a database.
// TODO: support default database.
if (namespace.length != 1) {
throw new NoSuchNamespaceException(namespace)
}
}

private def createOptionsWithTableName(ident: Identifier): JDBCOptions = {
// TODO: if table name contains special chars, we should quote it w.r.t. the JDBC dialect.
val tblName = (ident.namespace() :+ ident.name()).mkString(".")
new JDBCOptions(options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> tblName))
}

override def listTables(namespace: Array[String]): Array[Identifier] = {
// TODO: implement it when SHOW TABLES command support DS V2.
throw new UnsupportedOperationException("list table")
}

override def loadTable(ident: Identifier): Table = {
checkNamespace(ident.namespace())
val optionsWithTableName = createOptionsWithTableName(ident)
try {
val schema = JDBCRDD.resolveTable(optionsWithTableName)
JDBCTable(ident, schema, optionsWithTableName)
} catch {
case _: SQLException => throw new NoSuchTableException(ident)
}
}

override def createTable(
ident: Identifier,
schema: StructType,
partitions: Array[Transform],
properties: util.Map[String, String]): Table = {
if (!partitions.isEmpty) {
throw new UnsupportedOperationException("Cannot create JDBC table with partition")
}
// TODO: we can support this, but we need to add an API to `JdbcDialect` to generate the SQL
// statement to specify table options. Many options are not supported because of no table
// properties, e.g. the custom schema option, the partition column option, etc.
if (!properties.isEmpty) {
logWarning("Cannot create JDBC table with properties, these properties will be " +
"ignored: " + properties.asScala.map { case (k, v) => s"$k=$v" }.mkString("[", ", ", "]"))
}

val sb = new StringBuilder()
val dialect = JdbcDialects.get(options.url)
schema.fields.foreach { field =>
val name = dialect.quoteIdentifier(field.name)
val typ = JdbcUtils.getJdbcType(field.dataType, dialect).databaseTypeDefinition
val nullable = if (field.nullable) "" else "NOT NULL"
sb.append(s", $name $typ $nullable")
}
// TODO: support the `JDBC_CREATE_TABLE_COLUMN_TYPES` option, after we support table properties.
val schemaStr = if (sb.length < 2) "" else sb.substring(2)
val sql = s"CREATE TABLE $ident ($schemaStr)"
withConnection { conn =>
val statement = conn.createStatement
statement.setQueryTimeout(options.queryTimeout)
statement.executeUpdate(sql)
}

JDBCTable(ident, schema, createOptionsWithTableName(ident))
}

override def alterTable(ident: Identifier, changes: TableChange*): Table = {
// TODO: support this by adding more APIs to `JdbcDialect` which can generate the SQL statement
// to alter a table.
throw new UnsupportedOperationException("alter table")
}

override def dropTable(ident: Identifier): Boolean = {
try {
withConnection { conn =>
val statement = conn.createStatement
statement.setQueryTimeout(options.queryTimeout)
statement.executeUpdate(s"DROP TABLE $ident")
true
}
} catch {
case _: SQLException => false
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,15 @@

package org.apache.spark.sql.sources

import java.util

import org.apache.spark.annotation._
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalog.v2.TableCatalog
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.execution.streaming.{Sink, Source}
import org.apache.spark.sql.sources.v2.{Table, TableCapability}
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType

Expand Down Expand Up @@ -313,3 +316,27 @@ trait InsertableRelation {
trait CatalystScan {
def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row]
}

/**
* A special `TableCatalog` which returns `DataSourceV1Table`.
*
* @since 3.0.0
*/
@Experimental
@Unstable
trait DataSourceV1TableCatalog extends TableCatalog

/**
* A special Data Source V2 `Table`, which doesn't need to implement the read/write capabilities.
* Spark will fallback the read/write requests to the v1 relation.
Copy link
Contributor

Choose a reason for hiding this comment

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

consistent with upper case for wording V1 and V2?

*
* @since 3.0.0
*/
@Experimental
@Unstable
trait DataSourceV1Table extends Table {

override def capabilities(): util.Set[TableCapability] = util.Collections.emptySet()

def v1Relation: BaseRelation
}
Loading