-
Notifications
You must be signed in to change notification settings - Fork 3k
Spark: Validate table columns don't conflict with metadata columns #3456
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ | |
| import org.apache.iceberg.AssertHelpers; | ||
| import org.apache.iceberg.FileFormat; | ||
| import org.apache.iceberg.HasTableOperations; | ||
| import org.apache.iceberg.MetadataColumns; | ||
| import org.apache.iceberg.PartitionSpec; | ||
| import org.apache.iceberg.PartitionSpecParser; | ||
| import org.apache.iceberg.Schema; | ||
|
|
@@ -169,6 +170,35 @@ public void testPartitionMetadataColumnWithUnknownTransforms() { | |
| () -> sql("SELECT _partition FROM %s", TABLE_NAME)); | ||
| } | ||
|
|
||
| @Test | ||
| public void testConflictingColumns() { | ||
| table.updateSchema() | ||
| .addColumn(MetadataColumns.SPEC_ID.name(), Types.IntegerType.get()) | ||
| .addColumn(MetadataColumns.FILE_PATH.name(), Types.StringType.get()) | ||
| .commit(); | ||
|
Comment on lines
+175
to
+178
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Eventually, would it make sense to fail here, in the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe. I would like to avoid not allowing these names, but you're right that it would catch the problem earlier. |
||
|
|
||
| sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1', -1, 'path/to/file')", TABLE_NAME); | ||
|
|
||
| assertEquals("Rows must match", | ||
| ImmutableList.of(row(1L, "a1")), | ||
| sql("SELECT id, category FROM %s", TABLE_NAME)); | ||
|
|
||
| AssertHelpers.assertThrows("Should fail to query conflicting columns", | ||
| ValidationException.class, "column names conflict", | ||
| () -> sql("SELECT * FROM %s", TABLE_NAME)); | ||
|
|
||
| table.refresh(); | ||
|
|
||
| table.updateSchema() | ||
| .renameColumn(MetadataColumns.SPEC_ID.name(), "_renamed" + MetadataColumns.SPEC_ID.name()) | ||
| .renameColumn(MetadataColumns.FILE_PATH.name(), "_renamed" + MetadataColumns.FILE_PATH.name()) | ||
| .commit(); | ||
|
|
||
| assertEquals("Rows must match", | ||
| ImmutableList.of(row(0, null, -1)), | ||
| sql("SELECT _spec_id, _partition, _renamed_spec_id FROM %s", TABLE_NAME)); | ||
| } | ||
|
|
||
| private void createAndInitTable() throws IOException { | ||
| this.table = TestTables.create(temp.newFolder(), TABLE_NAME, SCHEMA, PartitionSpec.unpartitioned()); | ||
|
|
||
|
|
||
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 initially did this for Spark only. It may make sense to move it to
DataTableScanincore.