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

Support changing column types in Iceberg #15651

Merged
merged 5 commits into from
Jan 18, 2023

Conversation

ebyhr
Copy link
Member

@ebyhr ebyhr commented Jan 10, 2023

Description

Support changing column types in Iceberg
Relates to #15515

Release notes

(x) Release notes are required, with the following suggested text:

# Iceberg
* Add support for changing column types. ({issue}`15515`)

@cla-bot cla-bot bot added the cla-signed label Jan 10, 2023
@ebyhr ebyhr self-assigned this Jan 10, 2023
@ebyhr ebyhr force-pushed the ebi/iceberg-alter-column-set-data-type branch from 9979ddf to 3095a69 Compare January 10, 2023 10:15
@ebyhr ebyhr force-pushed the ebi/iceberg-alter-column-set-data-type branch 3 times, most recently from 7436d3a to 0925f61 Compare January 12, 2023 03:16
@ebyhr
Copy link
Member Author

ebyhr commented Jan 12, 2023

CI hit #15367

@ebyhr ebyhr force-pushed the ebi/iceberg-alter-column-set-data-type branch 2 times, most recently from 560fd7d to 3f8a2fc Compare January 16, 2023 05:42
@ebyhr
Copy link
Member Author

ebyhr commented Jan 16, 2023

Rebased on upstream to resolve conflicts and added some test cases.

assertThat(query("SELECT * FROM " + table.getName()))
.matches("VALUES bigint '123'");
assertThat((String) computeScalar("SHOW CREATE TABLE " + table.getName()))
.contains("partitioning = ARRAY['col']");
Copy link
Contributor

@findinpath findinpath Jan 16, 2023

Choose a reason for hiding this comment

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

Can we check that the partition predicate is fully pushed down (isFullyPushedDown() assertion) on a partition query?

@ebyhr ebyhr force-pushed the ebi/iceberg-alter-column-set-data-type branch from 3f8a2fc to e24d829 Compare January 17, 2023 03:35
@ebyhr
Copy link
Member Author

ebyhr commented Jan 17, 2023

@findepi Could you review this PR when you have time?

@ebyhr
Copy link
Member Author

ebyhr commented Jan 17, 2023

CI hit #13288. check-commits-dispatcher will be fixed in #15739

@@ -2295,7 +2295,7 @@ public void testSetColumnTypeWithComment()
{
skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT));

try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_comment_", "(col int COMMENT 'test')")) {
try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_comment_", "(col int COMMENT 'test comment')")) {
Copy link
Member

Choose a reason for hiding this comment

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

Was it a dead code until now?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes, it was a dead code because Postgres connector doesn't support creating a table with column comments.

@@ -2269,6 +2271,7 @@ private List<SetColumnTypeSetup> setColumnTypeSetupData()
.add(new SetColumnTypeSetup("char(100)", "'shorten-char'", "char(50)", "cast('shorten-char' as char(50))"))
.add(new SetColumnTypeSetup("char(100)", "'char-to-varchar'", "varchar", "'char-to-varchar'"))
.add(new SetColumnTypeSetup("varchar", "'varchar-to-char'", "char(100)", "cast('varchar-to-char' as char(100))"))
.add(new SetColumnTypeSetup("row(x int)", "row(1)", "row(y int)", "cast(row(1) as row(y int))"))
Copy link
Member

Choose a reason for hiding this comment

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

I am not sure this should be the expected behavior.
I would rather map row fields by name: the x field is gone, the y field is new, it should have a null value.

Copy link
Member Author

Choose a reason for hiding this comment

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

Changed to row(x int) -> row(x bigint) instead.

Table icebergTable = catalog.loadTable(session, table.getSchemaTableName());
try {
icebergTable.updateSchema()
.updateColumn(column.getName(), toIcebergType(type).asPrimitiveType())
Copy link
Member

Choose a reason for hiding this comment

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

What about allowing array(integer) to become array(bigint)
or row(...) to gain a new field?

Copy link
Member Author

Choose a reason for hiding this comment

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

We can't do that unless fixing Iceberg if my understanding is correct. Changing nested field types (e.g. row(x integer) -> row(x bigint)) in row columns is possible without changing Iceberg.

Comment on lines 6274 to 6277
if ((setup.sourceColumnType().equals("bigint") && setup.newColumnType().equals("integer")) ||
(setup.sourceColumnType().equals("decimal(5,3)") && setup.newColumnType().equals("decimal(5,2)")) ||
(setup.sourceColumnType().equals("varchar") && setup.newColumnType().equals("char(100)")) ||
(setup.sourceColumnType().equals("row(x int)") && setup.newColumnType().equals("row(y int)"))) {
Copy link
Member

Choose a reason for hiding this comment

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

That's subjective, but i find the following somewhat easier to read

switch ("%s -> %s".formatted(setup.sourceColumnType(), setup.newColumnType())) {
    case "bigint -> integer":
    case "decimal(5,3) -> decimal(5,2)":
    case "varchar -> char(100)":
    case "row(x int) -> row(y int)":
        // Iceberg allows updating column types if the update is safe. Safe updates are:
        // - int to bigint
        // - float to double
        // - decimal(P,S) to decimal(P2,S) when P2 > P (scale cannot change)
        // https://iceberg.apache.org/docs/latest/spark-ddl/#alter-table--alter-column
        return Optional.of(setup.asUnsupported());

    case "varchar(100) -> varchar(50)":
        // Iceberg connector ignores the varchar length
        return Optional.empty();
}

Copy link
Member

Choose a reason for hiding this comment

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

of course, this will become eve nicer when we can use record patterns in a switch (https://openjdk.org/jeps/405)

};

return Stream.of(StorageFormat.values())
.flatMap(storageFormat -> Arrays.stream(setColumnTypeData).map(data -> new Object[] {storageFormat, data[0], data[1], data[2], data[3]}))
Copy link
Member

Choose a reason for hiding this comment

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

Would DataProviders.cartesianProduct be applicable?

return cartesianProduct(
        Stream.of(StorageFormat.values())
                .collect(toDataProvider()),
        new Object[][] {
                {"integer", "2147483647", "bigint", 2147483647L},
                {"real", "10.3", "double", 10.3},
                {"real", "'NaN'", "double", Double.NaN},
                {"decimal(5,3)", "'12.345'", "decimal(10,3)", BigDecimal.valueOf(12.345)}
        });

@ebyhr ebyhr force-pushed the ebi/iceberg-alter-column-set-data-type branch from b2e432a to 2682f3a Compare January 18, 2023 07:16
@ebyhr
Copy link
Member Author

ebyhr commented Jan 18, 2023

Rebased on upstream to fix check-commits-dispatcher failures.

@ebyhr ebyhr force-pushed the ebi/iceberg-alter-column-set-data-type branch from 2682f3a to 45a3ac6 Compare January 18, 2023 08:41
@findepi
Copy link
Member

findepi commented Jan 18, 2023

see failures

@ebyhr
Copy link
Member Author

ebyhr commented Jan 18, 2023

CI hit #13779 & #15313. Both failures are unrelated to Iceberg.

@ebyhr ebyhr merged commit 6a3d3f8 into trinodb:master Jan 18, 2023
@ebyhr ebyhr deleted the ebi/iceberg-alter-column-set-data-type branch January 18, 2023 22:30
@ebyhr ebyhr mentioned this pull request Jan 18, 2023
@colebow colebow added this to the 406 milestone Jan 18, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Development

Successfully merging this pull request may close these issues.

6 participants