-
Notifications
You must be signed in to change notification settings - Fork 3k
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
Support changing column types in Iceberg #15651
Conversation
9979ddf
to
3095a69
Compare
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java
Show resolved
Hide resolved
...roduct-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java
Outdated
Show resolved
Hide resolved
7436d3a
to
0925f61
Compare
CI hit #15367 |
560fd7d
to
3f8a2fc
Compare
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']"); |
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.
Can we check that the partition predicate is fully pushed down (isFullyPushedDown()
assertion) on a partition query?
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java
Show resolved
Hide resolved
3f8a2fc
to
e24d829
Compare
@findepi Could you review this PR when you have time? |
@@ -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')")) { |
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.
Was it a dead code until now?
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.
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))")) |
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.
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.
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.
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()) |
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.
What about allowing array(integer)
to become array(bigint)
or row(...)
to gain a new field?
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.
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.
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)"))) { |
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.
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();
}
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.
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]})) |
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.
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)}
});
b2e432a
to
2682f3a
Compare
Rebased on upstream to fix |
2682f3a
to
45a3ac6
Compare
see failures |
Description
Support changing column types in Iceberg
Relates to #15515
Release notes
(x) Release notes are required, with the following suggested text: