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 @@ -103,22 +103,11 @@ object TableOutputResolver {
errors += _,
fillDefaultValue = supportColDefaultValue)
} else {
// If the target table needs more columns than the input query, fill them with
// the columns' default values, if the `supportColDefaultValue` parameter is true.
val fillDefaultValue = supportColDefaultValue && actualExpectedCols.size > query.output.size
val queryOutputCols = if (fillDefaultValue) {
query.output ++ actualExpectedCols.drop(query.output.size).flatMap { expectedCol =>
getDefaultValueExprOrNullLit(expectedCol, conf.useNullsForMissingDefaultColumnValues)
}
} else {
query.output
}
if (actualExpectedCols.size > queryOutputCols.size) {
if (actualExpectedCols.size > query.output.size) {
throw QueryCompilationErrors.cannotWriteNotEnoughColumnsToTableError(
tableName, actualExpectedCols.map(_.name), query)
}

resolveColumnsByPosition(tableName, queryOutputCols, actualExpectedCols, conf, errors += _)
resolveColumnsByPosition(tableName, query.output, actualExpectedCols, conf, errors += _)
}

if (errors.nonEmpty) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -404,7 +404,11 @@ object PreprocessTableInsertion extends ResolveInsertionBase {
}
val newQuery = try {
TableOutputResolver.resolveOutputColumns(
tblName, expectedColumns, query, byName = hasColumnList || insert.byName, conf,
tblName,
expectedColumns,
query,
byName = hasColumnList || insert.byName,
conf,
supportColDefaultValue = true)
} catch {
case e: AnalysisException if staticPartCols.nonEmpty &&
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,15 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession {

// INSERT without user-defined columns
sql("truncate table t")
sql("insert into t values (timestamp'2020-12-31')")
checkAnswer(spark.table("t"),
sql("select timestamp'2020-12-31', null").collect().head)
checkError(
exception = intercept[AnalysisException] {
sql("insert into t values (timestamp'2020-12-31')")
},
errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS",
parameters = Map(
"tableName" -> "`spark_catalog`.`default`.`t`",
"tableColumns" -> "`c1`, `c2`",
"dataColumns" -> "`col1`"))
}
}

Expand All @@ -57,18 +63,31 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession {

// INSERT without user-defined columns
sql("truncate table t")
sql("insert into t values (timestamp'2020-12-31')")
checkAnswer(spark.table("t"),
sql("select timestamp'2020-12-31', timestamp'2020-01-01'").collect().head)
checkError(
exception = intercept[AnalysisException] {
sql("insert into t values (timestamp'2020-12-31')")
},
errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS",
parameters = Map(
"tableName" -> "`spark_catalog`.`default`.`t`",
"tableColumns" -> "`c1`, `c2`",
"dataColumns" -> "`col1`"))
}
}

test("INSERT into partitioned tables") {
sql("create table t(c1 int, c2 int, c3 int, c4 int) using parquet partitioned by (c3, c4)")

// INSERT without static partitions
sql("insert into t values (1, 2, 3)")
checkAnswer(spark.table("t"), Row(1, 2, 3, null))
checkError(
exception = intercept[AnalysisException] {
sql("insert into t values (1, 2, 3)")
},
errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS",
parameters = Map(
"tableName" -> "`spark_catalog`.`default`.`t`",
"tableColumns" -> "`c1`, `c2`, `c3`, `c4`",
"dataColumns" -> "`col1`, `col2`, `col3`"))

// INSERT without static partitions but with column list
sql("truncate table t")
Expand All @@ -77,8 +96,16 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession {

// INSERT with static partitions
sql("truncate table t")
sql("insert into t partition(c3=3, c4=4) values (1)")
checkAnswer(spark.table("t"), Row(1, null, 3, 4))
checkError(
exception = intercept[AnalysisException] {
sql("insert into t partition(c3=3, c4=4) values (1)")
},
errorClass = "INSERT_PARTITION_COLUMN_ARITY_MISMATCH",
parameters = Map(
"tableName" -> "`spark_catalog`.`default`.`t`",
"tableColumns" -> "`c1`, `c2`, `c3`, `c4`",
"dataColumns" -> "`col1`",
"staticPartCols" -> "`c3`, `c4`"))

// INSERT with static partitions and with column list
sql("truncate table t")
Expand All @@ -87,8 +114,16 @@ class ResolveDefaultColumnsSuite extends QueryTest with SharedSparkSession {

// INSERT with partial static partitions
sql("truncate table t")
sql("insert into t partition(c3=3, c4) values (1, 2)")
checkAnswer(spark.table("t"), Row(1, 2, 3, null))
checkError(
exception = intercept[AnalysisException] {
sql("insert into t partition(c3=3, c4) values (1, 2)")
},
errorClass = "INSERT_PARTITION_COLUMN_ARITY_MISMATCH",
parameters = Map(
"tableName" -> "`spark_catalog`.`default`.`t`",
"tableColumns" -> "`c1`, `c2`, `c3`, `c4`",
"dataColumns" -> "`col1`, `col2`",
"staticPartCols" -> "`c3`"))

// INSERT with partial static partitions and with column list is not allowed
intercept[AnalysisException](sql("insert into t partition(c3=3, c4) (c1) values (1, 4)"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -962,11 +962,15 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
(1 to 10).map(i => Row(i, null))
)

sql("INSERT OVERWRITE TABLE jsonTable SELECT a FROM jt")
checkAnswer(
sql("SELECT a, b FROM jsonTable"),
(1 to 10).map(i => Row(i, null))
)
checkError(
exception = intercept[AnalysisException] {
sql("INSERT OVERWRITE TABLE jsonTable SELECT a FROM jt")
},
errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS",
parameters = Map(
"tableName" -> "`unknown`",
"tableColumns" -> "`a`, `b`",
"dataColumns" -> "`a`"))

sql("INSERT OVERWRITE TABLE jsonTable(a) SELECT a FROM jt")
checkAnswer(
Expand Down Expand Up @@ -1027,7 +1031,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
}
withTable("t") {
sql("create table t(i int, s bigint default 42, x bigint) using parquet")
sql("insert into t values(1)")
Copy link
Contributor

Choose a reason for hiding this comment

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

which Spark version added this test?

Copy link
Member Author

Choose a reason for hiding this comment

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

Added by #41262 which haven't been released yet.

sql("insert into t(i) values(1)")
checkAnswer(spark.table("t"), Row(1, 42L, null))
}
// The table has a partitioning column and a default value is injected.
Expand Down Expand Up @@ -1495,7 +1499,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
sql(createTableIntCol)
sql("alter table t add column s bigint default 42")
sql("alter table t add column x bigint")
sql("insert into t values(1)")
sql("insert into t(i) values(1)")
checkAnswer(spark.table("t"), Row(1, 42, null))
}
// The table has a partitioning column and a default value is injected.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -391,7 +391,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter
sql(s"INSERT INTO TABLE $tableName PARTITION (c=11, b=10) SELECT 9, 12")

// The data is missing a column. The default value for the missing column is null.
sql(s"INSERT INTO TABLE $tableName PARTITION (c=15, b=16) SELECT 13")
sql(s"INSERT INTO TABLE $tableName PARTITION (c=15, b=16) (a) SELECT 13")

// c is defined twice. Analyzer will complain.
intercept[ParseException] {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1258,11 +1258,11 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
"""INSERT INTO TABLE dp_test PARTITION(dp)
|SELECT key, value, key % 5 FROM src""".stripMargin)
},
errorClass = "_LEGACY_ERROR_TEMP_1169",
errorClass = "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS",
parameters = Map(
"tableName" -> "`spark_catalog`.`default`.`dp_test`",
"normalizedPartSpec" -> "dp",
"partColNames" -> "dp,sp"))
"tableColumns" -> "`key`, `value`, `dp`, `sp`",
"dataColumns" -> "`key`, `value`, `(key % 5)`"))

sql("SET hive.exec.dynamic.partition.mode=nonstrict")

Expand Down