Skip to content

Conversation

@cloud-fan
Copy link
Contributor

@cloud-fan cloud-fan commented May 22, 2023

What changes were proposed in this pull request?

This PR refactors the default column value resolution so that we don't need an extra DS v2 API for external v2 sources. The general idea is to split the default column value resolution into two parts:

  1. resolve the column "DEFAULT" to the column default expression. This applies to Project/UnresolvedInlineTable under InsertIntoStatement, and assignment expressions in UpdateTable/MergeIntoTable.
  2. fill missing columns with column default values for the input query. This does not apply to UPDATE and non-INSERT action of MERGE as they use the column from the target table as the default value.

The first part should be done for all the data sources, as it's part of column resolution. The second part should not be applied to v2 data sources with ACCEPT_ANY_SCHEMA, as they are free to define how to handle missing columns.

More concretely, this PR:

  1. put the column "DEFAULT" resolution logic in the rule ResolveReferences, with two new virtual rules. This is to follow [SPARK-41405][SQL] Centralize the column resolution logic #38888
  2. put the missing column handling in TableOutputResolver, which is shared by both the v1 and v2 insertion resolution rule. External v2 data sources can add custom catalyst rules to deal with missing columns for themselves.
  3. Remove the old rule ResolveDefaultColumns. Note that, with the refactor, we no long need to manually look up the table. We will deal with column default values after the target table of INSERT/UPDATE/MERGE is resolved.
  4. Remove the rule ResolveUserSpecifiedColumns and merge it to PreprocessTableInsertion. These two rules are both to resolve v1 insertion, and it's tricky to reason about their interactions. It's clearer to resolve the insertion with one pass.

Why are the changes needed?

code cleanup and remove unneeded DS v2 API.

Does this PR introduce any user-facing change?

No

How was this patch tested?

updated tests

Copy link
Member

Choose a reason for hiding this comment

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

nit: plan is always InsertIntoStatement from the caller side. I also noticed that the input of ResolveReferencesInUpdate is UpdateTable. Shall we make them consistent?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We will add v2 write commands later. I'll add a TODO here.

Copy link
Member

@gengliangwang gengliangwang May 23, 2023

Choose a reason for hiding this comment

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

ResolveColumnDefaultInInsert?

@gengliangwang
Copy link
Member

LGTM, pending on tests.
Awesome refactoring!

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Missing columns should not fail and we test it in ResolveDefaultColumnsSuite

Copy link
Contributor Author

Choose a reason for hiding this comment

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

MERGE/UPDATE are tested in Align[Update|Merge]AssignmentsSuite

@cloud-fan cloud-fan changed the title [WIP] refactor default column value resolution [SPARK-43742][SQL] Refactor default column value resolution May 23, 2023
Copy link
Member

Choose a reason for hiding this comment

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

If you don't mind, please file a JIRA and use the IDed TODO here.

object ResolveInsertInto extends Rule[LogicalPlan] {

/** Add a project to use the table column names for INSERT INTO BY NAME */
private def createProjectForByNameQuery(i: InsertIntoStatement): LogicalPlan = {
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 code here is unchanged but just moved to ResolveInsertionBase

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Unify the errors between v1 and v2 inserts.

Copy link
Contributor Author

@cloud-fan cloud-fan May 24, 2023

Choose a reason for hiding this comment

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

This change is needed. We want to resolve the table first, then resolve the column "DEFAULT" in the query. This means we can't wait for the query to be resolved before resolving the table.

Copy link
Contributor

Choose a reason for hiding this comment

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

sounds good, with any luck this can help reduce dependencies on rule orderings within the analyzer.

Copy link
Member

Choose a reason for hiding this comment

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

If we remove the pattern guard in this code, some operations on the "i.query" will fail later on. I create #44326 to fix

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It's fragile to resolve insert column list and static partitions separately. This PR resolves both in PreprocessTableInsertion for v1 insert. Spark already resolves both for v2 inserts in ResolveInsertInto.

Copy link
Contributor Author

@cloud-fan cloud-fan May 24, 2023

Choose a reason for hiding this comment

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

This is an existing issue that the default column value doesn't work for v2 inserts. I decided to fix it later as it needs to update quite some v2 tests.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is needed due to https://github.com/apache/spark/pull/41262/files#r1204075426 . Now it's possible that table is resolved but the query is not.

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 error was changed by 9f0bf51 and now it's restored. I think it's more accurate to report column already exists error rather than inline table error.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

now we create null literal with the expected data type directly.

Copy link
Contributor Author

@cloud-fan cloud-fan May 24, 2023

Choose a reason for hiding this comment

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

sql8 is s"UPDATE testcat.defaultvalues SET i=DEFAULT, s=DEFAULT WHERE i=DEFAULT"

I think it's almost impossible to find out all the improper places for hosting the column "DEFAULT", e.g. how about the UPDATE/MERGE assignment key? Other operators like Sort? This PR only checks the nested column "DEFAULT" and fails. If the column "DEFAULT" appears in improper places, we won't resolve it and users will hit unresolved column error.

Copy link
Contributor

Choose a reason for hiding this comment

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

This sounds reasonable, we can leave this test here, but we don't have to exhaustively cover all the cases.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

simplify the MERGE statement to focus on missing cols.

Copy link
Contributor Author

@cloud-fan cloud-fan May 24, 2023

Choose a reason for hiding this comment

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

it should be allowed. According to the tests the default value resolution is not triggered in some cases before this PR.

Copy link
Contributor

Choose a reason for hiding this comment

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

should we keep the test, but change its result to assert that it succeeds? or is this behavior exercised elsewhere in this test file?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

remove this conf setting as it's true by default.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It's more accurate to report that the partition column list in INSERT does not match the actual table partition columns.

Copy link
Contributor

@dtenedor dtenedor left a comment

Choose a reason for hiding this comment

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

Thanks for doing this! Took one initial review pass.

Copy link
Contributor

Choose a reason for hiding this comment

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

sounds good, with any luck this can help reduce dependencies on rule orderings within the analyzer.

def apply(plan: LogicalPlan): LogicalPlan = plan match {
case i: InsertIntoStatement if conf.enableDefaultColumns && i.table.resolved &&
i.query.containsPattern(UNRESOLVED_ATTRIBUTE) =>
val staticPartCols = i.partitionSpec.filter(_._2.isDefined).keys.map(normalizeFieldName).toSet
Copy link
Contributor

Choose a reason for hiding this comment

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

this is a bit hard to read, can we split the transformations into different lines with vals, and use an explicit name instead of _2 to refer to the column?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

InsertIntoStatement#partitionSpec is Map[String, Option[String]], and in Scala we can only use _2 to refer to the map value.

case i: InsertIntoStatement if conf.enableDefaultColumns && i.table.resolved &&
i.query.containsPattern(UNRESOLVED_ATTRIBUTE) =>
val staticPartCols = i.partitionSpec.filter(_._2.isDefined).keys.map(normalizeFieldName).toSet
val expectedQuerySchema = i.table.schema.filter { field =>
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 have a brief comment saying what this is?

case p: Project if acceptProject && p.child.resolved &&
p.containsPattern(UNRESOLVED_ATTRIBUTE) &&
p.projectList.length <= expectedQuerySchema.length =>
val newProjectList = p.projectList.zipWithIndex.map {
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 have some comment here describing the logic of adding new unresolved attributes referring to "DEFAULT" if the provided query has fewer columns than the target table, or else converting such existing unresolved attributes to their corresponding values?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'll add doc for the resolveColumnDefault method.

exprs.zipWithIndex.map {
case (u: UnresolvedAttribute, i) if isExplicitDefaultColumn(u) =>
val field = expectedQuerySchema(i)
getDefaultValueExpr(field).getOrElse(Literal(null, field.dataType))
Copy link
Contributor

Choose a reason for hiding this comment

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

we could integrate the Literal(null) part into getDefaultValueExpr since we want to use the NULL value if the default metadata is not present in every case. Or is this getDefaultValueExprOrNullLiteral, which we can use instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

There is a subtle difference: For missing cols, the default null value is optional (controlled by a flag). For the column "DEFAULT", it's a new feature when we add default value support and we can always use null as the default value if it's not defined.

Copy link
Contributor

Choose a reason for hiding this comment

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

optional: should we add a boolean argument to getDefaultValueExprOrNullLiteral to switch the behavior between the two modes?

resolvedKey match {
case attr: AttributeReference if conf.enableDefaultColumns =>
resolved match {
case u: UnresolvedAttribute if isExplicitDefaultColumn(u) =>
Copy link
Contributor

Choose a reason for hiding this comment

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

same, let's add a comment here mentioning that we're looking for unresolved attribute references to "DEFAULT" and replacing them?

Copy link
Contributor

@dtenedor dtenedor left a comment

Choose a reason for hiding this comment

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

Thanks for doing this! Took one initial review pass.

Copy link
Contributor

@dtenedor dtenedor left a comment

Choose a reason for hiding this comment

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

I reviewed carefully through the whole PR again, the logic and testing looks good. For any tables with capability ACCEPT_ANY_SCHEMA, we will bypass all this logic and the rest of the work is up to custom logic for those tables. We might have to duplicate some of this if any of those tables want to support default column values. But that sounds fair given the intended meaning of this capability.

p.projectList.length <= expectedQuerySchema.length =>
val newProjectList = p.projectList.zipWithIndex.map {
case (u: UnresolvedAttribute, i) if isExplicitDefaultColumn(u) =>
val field = expectedQuerySchema(i)
Copy link
Contributor

Choose a reason for hiding this comment

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

optional: when I wrote the original ResolveDefaultColumns rule, I named this variable insertTableSchemaWithoutPartitionColumns because I found myself confused frequently when reading the variable name. We could name this insertTargetTableSchema to clarify this, or insertTargetTableSchemaWithoutPartitionColumns if you don't think that's too verbose.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

tableSchema is not very accurate, and neither does insertTargetTableSchemaWithoutPartitionColumns. It's actually table schema excluding partition columns with static values.

That's why I choose expectedQuerySchema. People can read comments of the caller of this function to understand how we define the expected query schema.


case _: GlobalLimit | _: LocalLimit | _: Offset | _: Sort if acceptProject =>
plan.mapChildren(
resolveColumnDefault(_, expectedQuerySchema, acceptInlineTable = false))
Copy link
Contributor

Choose a reason for hiding this comment

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

optional: It looks like the only purpose of acceptInlineTable is setting it to false here in the event of a LIMIT and/or OFFSET and/or ORDER BY on top of a VALUES list. Do you think this check is strictly necessary? If not, we can simplify by removing acceptInlineTable as an argument to this function.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I don't think it's necessary but just want to keep the old behavior. Let me remove it.

exprs.zipWithIndex.map {
case (u: UnresolvedAttribute, i) if isExplicitDefaultColumn(u) =>
val field = expectedQuerySchema(i)
getDefaultValueExpr(field).getOrElse(Literal(null, field.dataType))
Copy link
Contributor

Choose a reason for hiding this comment

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

optional: should we add a boolean argument to getDefaultValueExprOrNullLiteral to switch the behavior between the two modes?

Copy link
Contributor

Choose a reason for hiding this comment

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

This sounds reasonable, we can leave this test here, but we don't have to exhaustively cover all the cases.

Copy link
Contributor

Choose a reason for hiding this comment

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

should we keep the test, but change its result to assert that it succeeds? or is this behavior exercised elsewhere in this test file?

assert(intercept[AnalysisException] {
sql("insert into t (i) values (default)")
}.getMessage.contains(addOneColButExpectedTwo))
}.getMessage.contains("Cannot find data for output column 's'"))
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 dedup this expected error message substring into one place, or even better, use checkError to assert on the error class?

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

+1, LGTM.

@dongjoon-hyun
Copy link
Member

Merged to master for Apache Spark 3.5.0.

czxm pushed a commit to czxm/spark that referenced this pull request Jun 12, 2023
### What changes were proposed in this pull request?

This PR refactors the default column value resolution so that we don't need an extra DS v2 API for external v2 sources. The general idea is to split the default column value resolution into two parts:
1. resolve the column "DEFAULT" to the column default expression. This applies to `Project`/`UnresolvedInlineTable` under `InsertIntoStatement`, and assignment expressions in `UpdateTable`/`MergeIntoTable`.
2. fill missing columns with column default values for the input query. This does not apply to UPDATE and non-INSERT action of MERGE as they use the column from the target table as the default value.

The first part should be done for all the data sources, as it's part of column resolution. The second part should not be applied to v2 data sources with `ACCEPT_ANY_SCHEMA`, as they are free to define how to handle missing columns.

More concretely, this PR:
1. put the column "DEFAULT" resolution logic in the rule `ResolveReferences`, with two new virtual rules. This is to follow apache#38888
2. put the missing column handling in `TableOutputResolver`, which is shared by both the v1 and v2 insertion resolution rule. External v2 data sources can add custom catalyst rules to deal with missing columns for themselves.
3. Remove the old rule `ResolveDefaultColumns`. Note that, with the refactor, we no long need to manually look up the table. We will deal with column default values after the target table of INSERT/UPDATE/MERGE is resolved.
4. Remove the rule `ResolveUserSpecifiedColumns` and merge it to `PreprocessTableInsertion`. These two rules are both to resolve v1 insertion, and it's tricky to reason about their interactions. It's clearer to resolve the insertion with one pass.
### Why are the changes needed?

code cleanup and remove unneeded DS v2 API.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

updated tests

Closes apache#41262 from cloud-fan/def-val.

Authored-by: Wenchen Fan <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
dongjoon-hyun pushed a commit that referenced this pull request Jun 21, 2023
… for non-nullable columns

### What changes were proposed in this pull request?

A followup of #41262 to fix a mistake. If a column has no default value and is not nullable, we should fail if people want to use its default value via the explicit `DEFAULT` name, and do not fill missing columns in INSERT.

### Why are the changes needed?

fix a wrong behavior

### Does this PR introduce _any_ user-facing change?

yes, otherwise the DML command will fail later at runtime.

### How was this patch tested?

new tests

Closes #41656 from cloud-fan/def-val.

Lead-authored-by: Wenchen Fan <[email protected]>
Co-authored-by: Wenchen Fan <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
MaxGekk added a commit that referenced this pull request Aug 29, 2023
### What changes were proposed in this pull request?
In the PR, I propose to raise an error when an user uses V1 `INSERT` without a list of columns, and the number of inserting columns doesn't match to the number of actual table columns.

At the moment Spark inserts data successfully in such case after the PR #41262 which changed the behaviour of Spark 3.4.x.

### Why are the changes needed?
1. To conform the SQL standard which requires the number of columns must be the same:
![Screenshot 2023-08-07 at 11 01 27 AM](https://github.com/apache/spark/assets/1580697/c55badec-5716-490f-a83a-0bb6b22c84c7)

Apparently, the insertion below must not succeed:
```sql
spark-sql (default)> CREATE TABLE tabtest(c1 INT, c2 INT);
spark-sql (default)> INSERT INTO tabtest SELECT 1;
```

2. To have the same behaviour as **Spark 3.4**:
```sql
spark-sql (default)> INSERT INTO tabtest SELECT 1;
`spark_catalog`.`default`.`tabtest` requires that the data to be inserted have the same number of columns as the target table: target table has 2 column(s) but the inserted data has 1 column(s), including 0 partition column(s) having constant value(s).
```

### Does this PR introduce _any_ user-facing change?
Yes.

After the changes:
```sql
spark-sql (default)> INSERT INTO tabtest SELECT 1;
[INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS] Cannot write to `spark_catalog`.`default`.`tabtest`, the reason is not enough data columns:
Table columns: `c1`, `c2`.
Data columns: `1`.
```

### How was this patch tested?
By running the modified tests:
```
$ build/sbt "test:testOnly *InsertSuite"
$ build/sbt "test:testOnly *ResolveDefaultColumnsSuite"
$ build/sbt -Phive "test:testOnly *HiveQuerySuite"
```

Closes #42393 from MaxGekk/fix-num-cols-insert.

Authored-by: Max Gekk <[email protected]>
Signed-off-by: Max Gekk <[email protected]>
MaxGekk added a commit that referenced this pull request Aug 29, 2023
### What changes were proposed in this pull request?
In the PR, I propose to raise an error when an user uses V1 `INSERT` without a list of columns, and the number of inserting columns doesn't match to the number of actual table columns.

At the moment Spark inserts data successfully in such case after the PR #41262 which changed the behaviour of Spark 3.4.x.

### Why are the changes needed?
1. To conform the SQL standard which requires the number of columns must be the same:
![Screenshot 2023-08-07 at 11 01 27 AM](https://github.com/apache/spark/assets/1580697/c55badec-5716-490f-a83a-0bb6b22c84c7)

Apparently, the insertion below must not succeed:
```sql
spark-sql (default)> CREATE TABLE tabtest(c1 INT, c2 INT);
spark-sql (default)> INSERT INTO tabtest SELECT 1;
```

2. To have the same behaviour as **Spark 3.4**:
```sql
spark-sql (default)> INSERT INTO tabtest SELECT 1;
`spark_catalog`.`default`.`tabtest` requires that the data to be inserted have the same number of columns as the target table: target table has 2 column(s) but the inserted data has 1 column(s), including 0 partition column(s) having constant value(s).
```

### Does this PR introduce _any_ user-facing change?
Yes.

After the changes:
```sql
spark-sql (default)> INSERT INTO tabtest SELECT 1;
[INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS] Cannot write to `spark_catalog`.`default`.`tabtest`, the reason is not enough data columns:
Table columns: `c1`, `c2`.
Data columns: `1`.
```

### How was this patch tested?
By running the modified tests:
```
$ build/sbt "test:testOnly *InsertSuite"
$ build/sbt "test:testOnly *ResolveDefaultColumnsSuite"
$ build/sbt -Phive "test:testOnly *HiveQuerySuite"
```

Closes #42393 from MaxGekk/fix-num-cols-insert.

Authored-by: Max Gekk <[email protected]>
Signed-off-by: Max Gekk <[email protected]>
(cherry picked from commit a7eef21)
Signed-off-by: Max Gekk <[email protected]>
addError(s"Cannot find data for output column '${newColPath.quoted}'")
None
val defaultExpr = if (fillDefaultValue) {
getDefaultValueExprOrNullLit(expectedCol, conf)
Copy link
Member

@yaooqinn yaooqinn Jul 3, 2024

Choose a reason for hiding this comment

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

This pollutes the expressions with unreplaced char/varchar and could result in bugs

yaooqinn added a commit that referenced this pull request Jul 5, 2024
… to a table with char/varchar

### What changes were proposed in this pull request?

#41262 introduced a regression by applying literals with char/varchar type in query output for table insertions, see

https://github.com/apache/spark/pull/41262/files#diff-6e331e8f1c67b5920fb46263b6e582ec6e6a253ee45543559c9692a72a1a40ecR187-R188

This causes bugs

```java
24/07/03 16:29:01 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
org.apache.spark.SparkException: [INTERNAL_ERROR] Unsupported data type VarcharType(64). SQLSTATE: XX000
	at org.apache.spark.SparkException$.internalError(SparkException.scala:92)
	at org.apache.spark.SparkException$.internalError(SparkException.scala:96)
```

```java
org.apache.spark.SparkUnsupportedOperationException: VarcharType(64) is not supported yet.
	at org.apache.spark.sql.errors.QueryExecutionErrors$.dataTypeUnsupportedYetError(QueryExecutionErrors.scala:993)
	at org.apache.spark.sql.execution.datasources.orc.OrcSerializer.newConverter(OrcSerializer.scala:209)
	at org.apache.spark.sql.execution.datasources.orc.OrcSerializer.$anonfun$converters$2(OrcSerializer.scala:35)
	at scala.collection.immutable.List.map(List.scala:247)
```

### Why are the changes needed?

Bugfix

### Does this PR introduce _any_ user-facing change?

no
### How was this patch tested?

new tests
### Was this patch authored or co-authored using generative AI tooling?
no

Closes #47198 from yaooqinn/SPARK-48792.

Authored-by: Kent Yao <[email protected]>
Signed-off-by: Kent Yao <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants