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-27845][SQL] DataSourceV2: InsertTable #24832

Closed
wants to merge 5 commits into from
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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -294,8 +294,8 @@ query
;

insertInto
: INSERT OVERWRITE TABLE tableIdentifier (partitionSpec (IF NOT EXISTS)?)? #insertOverwriteTable
| INSERT INTO TABLE? tableIdentifier partitionSpec? #insertIntoTable
: INSERT OVERWRITE TABLE? multipartIdentifier (partitionSpec (IF NOT EXISTS)?)? #insertOverwriteTable
| INSERT INTO TABLE? multipartIdentifier partitionSpec? (IF NOT EXISTS)? #insertIntoTable
Copy link
Contributor

Choose a reason for hiding this comment

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

do we need to wrap with parentheses (partitionSpec (IF NOT EXISTS)?)? like above? Otherwise, what happens if there's no partitionSpec but the IF NOT EXISTS?

  • If the table not exists? Then wouldn't that be CTAS?

Copy link
Contributor

Choose a reason for hiding this comment

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

It isn't supported either way, so why combine the two?

Copy link
Contributor

Choose a reason for hiding this comment

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

got it

Copy link
Contributor

@cloud-fan cloud-fan Jul 25, 2019

Choose a reason for hiding this comment

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

I think (partitionSpec (IF NOT EXISTS)?)? is better? INSERT INTO TABLE ... IF NOT EXISTS doesn't make sense. The IF NOT EXISTS is only for partitions.

Copy link
Contributor

Choose a reason for hiding this comment

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

Actually, IF NOT EXISTS doesn't make sense to partition either. It's append not overwrite, and it seems weird to me if we can't append to an existing partition.

Copy link
Contributor

Choose a reason for hiding this comment

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

Can we keep this unchanged and not add IF NOT EXISTS here?

Copy link
Contributor

Choose a reason for hiding this comment

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

This was changed to get a better error message. Instead of a parse exception that lists symbols, this is now a useful error message with a test.

| INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir
| INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList)? #insertOverwriteDir
;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@ import scala.util.Random

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, LookupCatalog, TableChange}
import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, IdentityTransform}
import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util.loadTable
import org.apache.spark.sql.catalyst._
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.encoders.OuterScopes
Expand All @@ -34,12 +36,14 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.expressions.objects._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement}
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, InsertIntoStatement}
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.catalyst.trees.TreeNodeRef
import org.apache.spark.sql.catalyst.util.toPrettySQL
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode
import org.apache.spark.sql.sources.v2.Table
import org.apache.spark.sql.types._

/**
Expand Down Expand Up @@ -167,6 +171,7 @@ class Analyzer(
Batch("Resolution", fixedPoint,
ResolveTableValuedFunctions ::
ResolveAlterTable ::
ResolveInsertInto ::
ResolveTables ::
ResolveRelations ::
ResolveReferences ::
Expand Down Expand Up @@ -757,6 +762,136 @@ class Analyzer(
}
}

object ResolveInsertInto extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case i @ InsertIntoStatement(
UnresolvedRelation(CatalogObjectIdentifier(Some(tableCatalog), ident)), _, _, _, _)
if i.query.resolved =>
loadTable(tableCatalog, ident)
.map(DataSourceV2Relation.create)
.map(relation => {
// ifPartitionNotExists is append with validation, but validation is not supported
if (i.ifPartitionNotExists) {
throw new AnalysisException(
s"Cannot write, IF NOT EXISTS is not supported for table: ${relation.table.name}")
}

val partCols = partitionColumnNames(relation.table)
validatePartitionSpec(partCols, i.partitionSpec)

val staticPartitions = i.partitionSpec.filter(_._2.isDefined).mapValues(_.get)
val query = addStaticPartitionColumns(relation, i.query, staticPartitions)
val dynamicPartitionOverwrite = partCols.size > staticPartitions.size &&
conf.partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC

if (!i.overwrite) {
AppendData.byPosition(relation, query)
} else if (dynamicPartitionOverwrite) {
OverwritePartitionsDynamic.byPosition(relation, query)
} else {
OverwriteByExpression.byPosition(
relation, query, staticDeleteExpression(relation, staticPartitions))
}
})
.getOrElse(i)

case i @ InsertIntoStatement(UnresolvedRelation(AsTableIdentifier(_)), _, _, _, _)
if i.query.resolved =>
InsertIntoTable(i.table, i.partitionSpec, i.query, i.overwrite, i.ifPartitionNotExists)
}

private def partitionColumnNames(table: Table): Seq[String] = {
// get partition column names. in v2, partition columns are columns that are stored using an
// identity partition transform because the partition values and the column values are
// identical. otherwise, partition values are produced by transforming one or more source
// columns and cannot be set directly in a query's PARTITION clause.
table.partitioning.flatMap {
case IdentityTransform(FieldReference(Seq(name))) => Some(name)
case _ => None
}
}

private def validatePartitionSpec(
partitionColumnNames: Seq[String],
partitionSpec: Map[String, Option[String]]): Unit = {
// check that each partition name is a partition column. otherwise, it is not valid
partitionSpec.keySet.foreach { partitionName =>
partitionColumnNames.find(name => conf.resolver(name, partitionName)) match {
case Some(_) =>
case None =>
throw new AnalysisException(
s"PARTITION clause cannot contain a non-partition column name: $partitionName")
}
}
}

private def addStaticPartitionColumns(
relation: DataSourceV2Relation,
query: LogicalPlan,
staticPartitions: Map[String, String]): LogicalPlan = {

if (staticPartitions.isEmpty) {
query

} else {
// add any static value as a literal column
val withStaticPartitionValues = {
// for each static name, find the column name it will replace and check for unknowns.
val outputNameToStaticName = staticPartitions.keySet.map(staticName =>
relation.output.find(col => conf.resolver(col.name, staticName)) match {
case Some(attr) =>
attr.name -> staticName
case _ =>
throw new AnalysisException(
s"Cannot add static value for unknown column: $staticName")
}).toMap

val queryColumns = query.output.iterator

// for each output column, add the static value as a literal, or use the next input
// column. this does not fail if input columns are exhausted and adds remaining columns
// at the end. both cases will be caught by ResolveOutputRelation and will fail the
// query with a helpful error message.
relation.output.flatMap { col =>
outputNameToStaticName.get(col.name).flatMap(staticPartitions.get) match {
case Some(staticValue) =>
Some(Alias(Cast(Literal(staticValue), col.dataType), col.name)())
case _ if queryColumns.hasNext =>
Some(queryColumns.next)
case _ =>
None
}
} ++ queryColumns
}

Project(withStaticPartitionValues, query)
}
}

private def staticDeleteExpression(
relation: DataSourceV2Relation,
staticPartitions: Map[String, String]): Expression = {
if (staticPartitions.isEmpty) {
Literal(true)
} else {
staticPartitions.map { case (name, value) =>
relation.output.find(col => conf.resolver(col.name, name)) match {
case Some(attr) =>
// the delete expression must reference the table's column names, but these attributes
// are not available when CheckAnalysis runs because the relation is not a child of
// the logical operation. instead, expressions are resolved after
// ResolveOutputRelation runs, using the query's column names that will match the
// table names at that point. because resolution happens after a future rule, create
// an UnresolvedAttribute.
EqualTo(UnresolvedAttribute(attr.name), Cast(Literal(value), attr.dataType))
case None =>
throw new AnalysisException(s"Unknown static partition column: $name")
}
}.reduce(And)
}
}
}

/**
* Resolve ALTER TABLE statements that use a DSv2 catalog.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.expressions.objects.Invoke
import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.logical.sql._
import org.apache.spark.sql.types._

/**
Expand Down Expand Up @@ -379,10 +380,14 @@ package object dsl {
Generate(generator, unrequiredChildIndex, outer,
alias, outputNames.map(UnresolvedAttribute(_)), logicalPlan)

def insertInto(tableName: String, overwrite: Boolean = false): LogicalPlan =
InsertIntoTable(
analysis.UnresolvedRelation(TableIdentifier(tableName)),
Map.empty, logicalPlan, overwrite, ifPartitionNotExists = false)
def insertInto(tableName: String): LogicalPlan = insertInto(table(tableName))

def insertInto(
table: LogicalPlan,
partition: Map[String, Option[String]] = Map.empty,
overwrite: Boolean = false,
ifPartitionNotExists: Boolean = false): LogicalPlan =
InsertIntoStatement(table, partition, logicalPlan, overwrite, ifPartitionNotExists)

def as(alias: String): LogicalPlan = SubqueryAlias(alias, logicalPlan)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last}
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement}
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement}
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -239,9 +239,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging

/**
* Parameters used for writing query to a table:
* (tableIdentifier, partitionKeys, exists).
* (multipartIdentifier, partitionKeys, ifPartitionNotExists).
*/
type InsertTableParams = (TableIdentifier, Map[String, Option[String]], Boolean)
type InsertTableParams = (Seq[String], Map[String, Option[String]], Boolean)

/**
* Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider).
Expand All @@ -263,11 +263,21 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
ctx match {
case table: InsertIntoTableContext =>
val (tableIdent, partitionKeys, exists) = visitInsertIntoTable(table)
InsertIntoTable(UnresolvedRelation(tableIdent), partitionKeys, query, false, exists)
val (tableIdent, partition, ifPartitionNotExists) = visitInsertIntoTable(table)
InsertIntoStatement(
UnresolvedRelation(tableIdent),
partition,
query,
overwrite = false,
ifPartitionNotExists)
case table: InsertOverwriteTableContext =>
val (tableIdent, partitionKeys, exists) = visitInsertOverwriteTable(table)
InsertIntoTable(UnresolvedRelation(tableIdent), partitionKeys, query, true, exists)
val (tableIdent, partition, ifPartitionNotExists) = visitInsertOverwriteTable(table)
InsertIntoStatement(
UnresolvedRelation(tableIdent),
partition,
query,
overwrite = true,
ifPartitionNotExists)
case dir: InsertOverwriteDirContext =>
val (isLocal, storage, provider) = visitInsertOverwriteDir(dir)
InsertIntoDir(isLocal, storage, provider, query, overwrite = true)
Expand All @@ -284,9 +294,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
*/
override def visitInsertIntoTable(
ctx: InsertIntoTableContext): InsertTableParams = withOrigin(ctx) {
val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
val tableIdent = visitMultipartIdentifier(ctx.multipartIdentifier)
val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty)

if (ctx.EXISTS != null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

what's the point of adding this to the parser, if we're not going to support it?

Copy link
Contributor

Choose a reason for hiding this comment

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

For a better error message that is testable. Before, there were no tests for this case and the error message listed expected symbols.

Copy link
Contributor

Choose a reason for hiding this comment

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

Also, since the PARTITION clause is optional for the above case, it shouldn't group the two together either. It is semantically incorrect because a write to a partitioned table is always a partitioned write.

operationNotAllowed("INSERT INTO ... IF NOT EXISTS", ctx)
}

(tableIdent, partitionKeys, false)
}

Expand All @@ -296,13 +310,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
override def visitInsertOverwriteTable(
ctx: InsertOverwriteTableContext): InsertTableParams = withOrigin(ctx) {
assert(ctx.OVERWRITE() != null)
val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
val tableIdent = visitMultipartIdentifier(ctx.multipartIdentifier)
Copy link
Contributor

Choose a reason for hiding this comment

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

I think this needs to be updated to remove the ParseException thrown when IF NOT EXISTS is present and there are dynamic partitions. I think that is an analysis problem, not a parse problem.

Also, I don't see a reason why IF NOT EXISTS would not be supported with dynamic partitions. Wouldn't that fail if any partitions would be overwritten? It seems to make sense to me, but maybe there is a good reason why this is not allowed? @gatorsmile can you comment?

Copy link
Contributor

Choose a reason for hiding this comment

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

We discussed this in the DSv2 sync last night and decided to add a method to the write builder to pass this IF NOT EXISTS flag. This will be done in a follow-up to avoid over-complicating this commit.

Copy link
Contributor

Choose a reason for hiding this comment

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

val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty)

val dynamicPartitionKeys: Map[String, Option[String]] = partitionKeys.filter(_._2.isEmpty)
if (ctx.EXISTS != null && dynamicPartitionKeys.nonEmpty) {
throw new ParseException(s"Dynamic partitions do not support IF NOT EXISTS. Specified " +
"partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx)
operationNotAllowed("IF NOT EXISTS with dynamic partitions: " +
Copy link
Contributor

Choose a reason for hiding this comment

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

why do we change the error message here?

Copy link
Contributor

Choose a reason for hiding this comment

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

This uses operationNotAllowed instead of throwing a custom ParseException, like other methods that do not allow specific combinations. I think it's a good idea to standardize on the existing method. This also makes the error message like the others, where it states clear what is not allowed.

dynamicPartitionKeys.keys.mkString(", "), ctx)
}

(tableIdent, partitionKeys, ctx.EXISTS() != null)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.catalyst.plans.logical.sql

import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan

/**
* An INSERT INTO statement, as parsed from SQL.
*
* @param table the logical plan representing the table.
* @param query the logical plan representing data to write to.
* @param overwrite overwrite existing table or partitions.
* @param partitionSpec a map from the partition key to the partition value (optional).
* If the value is missing, dynamic partition insert will be performed.
* As an example, `INSERT INTO tbl PARTITION (a=1, b=2) AS` would have
* Map('a' -> Some('1'), 'b' -> Some('2')),
* and `INSERT INTO tbl PARTITION (a=1, b) AS ...`
* would have Map('a' -> Some('1'), 'b' -> None).
* @param ifPartitionNotExists If true, only write if the partition does not exist.
* Only valid for static partitions.
*/
case class InsertIntoStatement(
table: LogicalPlan,
partitionSpec: Map[String, Option[String]],
query: LogicalPlan,
overwrite: Boolean,
ifPartitionNotExists: Boolean) extends ParsedStatement {

require(overwrite || !ifPartitionNotExists,
"IF NOT EXISTS is only valid in INSERT OVERWRITE")
require(partitionSpec.values.forall(_.nonEmpty) || !ifPartitionNotExists,
"IF NOT EXISTS is only valid with static partitions")

override def children: Seq[LogicalPlan] = query :: Nil
}