-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-16036][SPARK-16037][SQL] fix various table insertion problems #13754
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,7 +21,7 @@ import scala.util.control.NonFatal | |
|
|
||
| import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} | ||
| import org.apache.spark.sql.catalyst.analysis._ | ||
| import org.apache.spark.sql.catalyst.catalog.SessionCatalog | ||
| import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, SessionCatalog} | ||
| import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, RowOrdering} | ||
| import org.apache.spark.sql.catalyst.plans.logical | ||
| import org.apache.spark.sql.catalyst.plans.logical._ | ||
|
|
@@ -62,53 +62,79 @@ private[sql] class ResolveDataSource(sparkSession: SparkSession) extends Rule[Lo | |
| } | ||
|
|
||
| /** | ||
| * A rule to do pre-insert data type casting and field renaming. Before we insert into | ||
| * an [[InsertableRelation]], we will use this rule to make sure that | ||
| * the columns to be inserted have the correct data type and fields have the correct names. | ||
| * Preprocess the [[InsertIntoTable]] plan. Throws exception if the number of columns mismatch, or | ||
| * specified partition columns are different from the existing partition columns in the target | ||
| * table. It also does data type casting and field renaming, to make sure that the columns to be | ||
| * inserted have the correct data type and fields have the correct names. | ||
| */ | ||
| private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { | ||
| def apply(plan: LogicalPlan): LogicalPlan = plan transform { | ||
| // Wait until children are resolved. | ||
| case p: LogicalPlan if !p.childrenResolved => p | ||
|
|
||
| // We are inserting into an InsertableRelation or HadoopFsRelation. | ||
| case i @ InsertIntoTable( | ||
| l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _), _, child, _, _) => | ||
| // First, make sure the data to be inserted have the same number of fields with the | ||
| // schema of the relation. | ||
| if (l.output.size != child.output.size) { | ||
| sys.error( | ||
| s"$l requires that the data to be inserted have the same number of columns as the " + | ||
| s"target table: target table has ${l.output.size} column(s) but " + | ||
| s"the inserted data has ${child.output.size} column(s).") | ||
| } | ||
| castAndRenameChildOutput(i, l.output, child) | ||
| private[sql] object PreprocessTableInsertion extends Rule[LogicalPlan] { | ||
| private def preprocess( | ||
| insert: InsertIntoTable, | ||
| tblName: String, | ||
| partColNames: Seq[String]): InsertIntoTable = { | ||
|
|
||
| val expectedColumns = insert.expectedColumns | ||
| if (expectedColumns.isDefined && expectedColumns.get.length != insert.child.schema.length) { | ||
| throw new AnalysisException( | ||
| s"Cannot insert into table $tblName because the number of columns are different: " + | ||
| s"need ${expectedColumns.get.length} columns, " + | ||
| s"but query has ${insert.child.schema.length} columns.") | ||
| } | ||
|
|
||
| if (insert.partition.nonEmpty) { | ||
| // the query's partitioning must match the table's partitioning | ||
| // this is set for queries like: insert into ... partition (one = "a", two = <expr>) | ||
| if (insert.partition.keySet != partColNames.toSet) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This check is case-sensitive. |
||
| throw new AnalysisException( | ||
| s""" | ||
| |Requested partitioning does not match the table $tblName: | ||
| |Requested partitions: ${insert.partition.keys.mkString(",")} | ||
| |Table partitions: ${partColNames.mkString(",")} | ||
| """.stripMargin) | ||
| } | ||
| expectedColumns.map(castAndRenameChildOutput(insert, _)).getOrElse(insert) | ||
| } else { | ||
| // All partition columns are dynamic because this InsertIntoTable had no partitioning | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| expectedColumns.map(castAndRenameChildOutput(insert, _)).getOrElse(insert) | ||
| .copy(partition = partColNames.map(_ -> None).toMap) | ||
| } | ||
| } | ||
|
|
||
| /** If necessary, cast data types and rename fields to the expected types and names. */ | ||
| // TODO: do we really need to rename? | ||
| def castAndRenameChildOutput( | ||
| insertInto: InsertIntoTable, | ||
| expectedOutput: Seq[Attribute], | ||
| child: LogicalPlan): InsertIntoTable = { | ||
| val newChildOutput = expectedOutput.zip(child.output).map { | ||
| insert: InsertIntoTable, | ||
| expectedOutput: Seq[Attribute]): InsertIntoTable = { | ||
| val newChildOutput = expectedOutput.zip(insert.child.output).map { | ||
| case (expected, actual) => | ||
| val needCast = !expected.dataType.sameType(actual.dataType) | ||
| // We want to make sure the filed names in the data to be inserted exactly match | ||
| // names in the schema. | ||
| val needRename = expected.name != actual.name | ||
| (needCast, needRename) match { | ||
| case (true, _) => Alias(Cast(actual, expected.dataType), expected.name)() | ||
| case (false, true) => Alias(actual, expected.name)() | ||
| case (_, _) => actual | ||
| if (expected.dataType.sameType(actual.dataType) && expected.name == actual.name) { | ||
| actual | ||
| } else { | ||
| Alias(Cast(actual, expected.dataType), expected.name)() | ||
| } | ||
| } | ||
|
|
||
| if (newChildOutput == child.output) { | ||
| insertInto | ||
| if (newChildOutput == insert.child.output) { | ||
| insert | ||
| } else { | ||
| insertInto.copy(child = Project(newChildOutput, child)) | ||
| insert.copy(child = Project(newChildOutput, insert.child)) | ||
| } | ||
| } | ||
|
|
||
| def apply(plan: LogicalPlan): LogicalPlan = plan transform { | ||
| case i @ InsertIntoTable(table, partition, child, _, _) if table.resolved && child.resolved => | ||
| table match { | ||
| case relation: CatalogRelation => | ||
| val metadata = relation.catalogTable | ||
| preprocess(i, metadata.identifier.quotedString, metadata.partitionColumnNames) | ||
| case LogicalRelation(h: HadoopFsRelation, _, identifier) => | ||
| val tblName = identifier.map(_.quotedString).getOrElse("unknown") | ||
| preprocess(i, tblName, h.partitionSchema.map(_.name)) | ||
| case LogicalRelation(_: InsertableRelation, _, identifier) => | ||
| val tblName = identifier.map(_.quotedString).getOrElse("unknown") | ||
| preprocess(i, tblName, Nil) | ||
| case other => i | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -325,27 +325,6 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef | |
| } | ||
| } | ||
|
|
||
| test("Detect table partitioning with correct partition order") { | ||
| withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { | ||
| sql("CREATE TABLE source (id bigint, part2 string, part1 string, data string)") | ||
| val data = (1 to 10).map(i => (i, if ((i % 2) == 0) "even" else "odd", "p", s"data-$i")) | ||
| .toDF("id", "part2", "part1", "data") | ||
|
|
||
| data.write.insertInto("source") | ||
| checkAnswer(sql("SELECT * FROM source"), data.collect().toSeq) | ||
|
|
||
| // the original data with part1 and part2 at the end | ||
| val expected = data.select("id", "data", "part1", "part2") | ||
|
|
||
| sql( | ||
| """CREATE TABLE partitioned (id bigint, data string) | ||
| |PARTITIONED BY (part1 string, part2 string)""".stripMargin) | ||
| spark.table("source").write.insertInto("partitioned") | ||
|
|
||
| checkAnswer(sql("SELECT * FROM partitioned"), expected.collect().toSeq) | ||
| } | ||
| } | ||
|
|
||
| private def testPartitionedHiveSerDeTable(testName: String)(f: String => Unit): Unit = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it is better have this test. We can change the expected answers to respect the semantic.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nvm. We do not need this test anymore because the semantic has been changed (we will not adjust the column ordering). |
||
| test(s"Hive SerDe table - $testName") { | ||
| val hiveTable = "hive_table" | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -348,6 +348,7 @@ abstract class HiveComparisonTest | |
| queryString.replace("../../data", testDataPath)) | ||
| val containsCommands = originalQuery.analyzed.collectFirst { | ||
| case _: Command => () | ||
| case _: InsertIntoTable => () | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We should not have
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ah, it's because I removed the So adding a case here is a reasonable fix. |
||
| case _: LogicalInsertIntoHiveTable => () | ||
| }.nonEmpty | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Seems this
containsdoes not work for case-insensitive resolution. We can fix is in a separate PR.