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 @@ -3409,7 +3409,9 @@ class Analyzer(override val catalogManager: CatalogManager)
tableOutput: Seq[Attribute],
cols: Seq[NamedExpression],
query: LogicalPlan): LogicalPlan = {
if (cols.size != query.output.size) {
// No need to check column size when USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES is enabled,
// since all omitted column(s) will be added back to query automatically during parse.
if (!conf.useNullsForMissingDefaultColumnValues && cols.size != query.output.size) {
throw QueryCompilationErrors.writeTableWithMismatchedColumnsError(
cols.size, query.output.size, query)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,15 +36,17 @@ object TableOutputResolver {
byName: Boolean,
conf: SQLConf): LogicalPlan = {

if (expected.size < query.output.size) {
// No need to check column size when USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES is enabled,
// since all omitted column(s) will be added back to query automatically during parse.
if (!conf.useNullsForMissingDefaultColumnValues && expected.size < query.output.size) {
throw QueryCompilationErrors.cannotWriteTooManyColumnsToTableError(tableName, expected, query)
}

val errors = new mutable.ArrayBuffer[String]()
val resolved: Seq[NamedExpression] = if (byName) {
reorderColumnsByName(query.output, expected, conf, errors += _)
} else {
if (expected.size > query.output.size) {
if (!conf.useNullsForMissingDefaultColumnValues && expected.size > query.output.size) {
throw QueryCompilationErrors.cannotWriteNotEnoughColumnsToTableError(
tableName, expected, query)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -381,7 +381,10 @@ object PreprocessTableInsertion extends Rule[LogicalPlan] {
val staticPartCols = normalizedPartSpec.filter(_._2.isDefined).keySet
val expectedColumns = insert.table.output.filterNot(a => staticPartCols.contains(a.name))

if (expectedColumns.length != insert.query.schema.length) {
// No need to check column size when USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES is enabled,
// since all omitted column(s) will be added back to query automatically during parse.
if (!conf.useNullsForMissingDefaultColumnValues &&
expectedColumns.length != insert.query.schema.length) {
throw QueryCompilationErrors.mismatchedInsertedDataColumnNumberError(
tblName, insert, staticPartCols)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,26 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
)
}

test("SPARK-38707 Allow user to insert into only certain columns of a table") {
withSQLConf(SQLConf.USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES.key -> "true") {
withTable("t") {
sql("create table t(i boolean, s bigint) using parquet")
sql("insert into t(i) values(true)")
checkAnswer(sql("select i, s from t"), Seq(Row(true, null)))
}
}

withSQLConf(SQLConf.USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES.key -> "false") {
withTable("t") {
sql("create table t(i boolean, s bigint) using parquet")
assert(intercept[AnalysisException] {
sql("insert into t(i) values(true)")
}.getMessage.contains("requires that the data to be inserted have the same number of " +
"columns as the target"))
}
}
}

test("insert into a temp view that does not point to an insertable data source") {
import testImplicits._
withTempView("t1", "t2") {
Expand Down