-
Notifications
You must be signed in to change notification settings - Fork 72
Add multi schema update support for tables during replica table commits #407
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
Merged
Will-Lo
merged 11 commits into
linkedin:main
from
Will-Lo:add-multi-schema-update-support-tables-api
Dec 22, 2025
+341
−20
Merged
Changes from 9 commits
Commits
Show all changes
11 commits
Select commit
Hold shift + click to select a range
d47a7e5
Modify CreateTable/UpdateTable endpoint to support multi schema updat…
Will-Lo c556400
Add tests
Will-Lo 3bf22ba
Improve tests
Will-Lo 05f786d
Cleanup
Will-Lo d366bce
Avoid using snapshot serializer for schemas
Will-Lo 402e2b6
Address reviews initial
Will-Lo 96f10a2
Move tests, fix compilation issues
Will-Lo 76c074b
Fix tests
Will-Lo 36a406f
Ensure schema transformation avoids rebuilding metadata every time
Will-Lo d6fd211
Cleanup, add more tests
Will-Lo 2fce5db
Address review
Will-Lo File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
150 changes: 150 additions & 0 deletions
150
...test/java/com/linkedin/openhouse/spark/catalogtest/e2e/SparkMultiSchemaEvolutionTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,150 @@ | ||
| package com.linkedin.openhouse.spark.catalogtest.e2e; | ||
|
|
||
| import com.linkedin.openhouse.javaclient.OpenHouseCatalog; | ||
| import com.linkedin.openhouse.tablestest.OpenHouseSparkITest; | ||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
| import org.apache.iceberg.CatalogUtil; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.SchemaParser; | ||
| import org.apache.iceberg.TableMetadata; | ||
| import org.apache.iceberg.TableOperations; | ||
| import org.apache.iceberg.catalog.Catalog; | ||
| import org.apache.iceberg.catalog.TableIdentifier; | ||
| import org.apache.iceberg.types.Types; | ||
| import org.apache.spark.sql.SparkSession; | ||
| import org.junit.jupiter.api.Assertions; | ||
| import org.junit.jupiter.api.Test; | ||
| import scala.collection.JavaConverters; | ||
|
|
||
| public class SparkMultiSchemaEvolutionTest extends OpenHouseSparkITest { | ||
|
|
||
| @Test | ||
| void testMultiSchemaEvolution() throws Exception { | ||
| SparkSession spark = null; | ||
| try { | ||
| spark = getSparkSession(); | ||
| spark.sql( | ||
| "CREATE TABLE openhouse.multiSchemaTest.t1 (name string, id int) TBLPROPERTIES ('openhouse.tableType' = 'REPLICA_TABLE');"); | ||
| spark.sql("INSERT INTO openhouse.multiSchemaTest.t1 VALUES ('Alice', 1)"); | ||
| spark.sql("INSERT INTO openhouse.multiSchemaTest.t1 VALUES ('Bob', 2), ('Charlie', 3)"); | ||
| TableIdentifier tableIdentifier = TableIdentifier.of("multiSchemaTest", "t1"); | ||
| OpenHouseCatalog ohCatalog = (OpenHouseCatalog) getOpenHouseCatalog(spark); | ||
| TableOperations ops = ohCatalog.newTableOps(tableIdentifier); | ||
| Schema evolvedSchema = | ||
| new Schema( | ||
| Types.NestedField.optional(1, "name", Types.StringType.get()), | ||
| Types.NestedField.optional(2, "id", Types.IntegerType.get()), | ||
| Types.NestedField.optional(3, "newCol", Types.IntegerType.get())); | ||
| Schema finalEvolvedSchema = | ||
| new Schema( | ||
| Types.NestedField.optional(1, "name", Types.StringType.get()), | ||
| Types.NestedField.optional(2, "id", Types.IntegerType.get()), | ||
| Types.NestedField.optional(3, "newCol1", Types.IntegerType.get()), | ||
| Types.NestedField.optional(4, "newCol2", Types.IntegerType.get())); | ||
|
|
||
| TableMetadata metadata = ops.current(); | ||
| TableMetadata evolvedMetadata = | ||
| TableMetadata.buildFrom(metadata) | ||
| .addSchema(evolvedSchema, evolvedSchema.highestFieldId()) | ||
| .build(); | ||
| TableMetadata finalEvolvedMetadata = | ||
| TableMetadata.buildFrom(evolvedMetadata) | ||
| .addSchema(finalEvolvedSchema, finalEvolvedSchema.highestFieldId()) | ||
| .setCurrentSchema(2) | ||
| .build(); | ||
|
|
||
| Assertions.assertEquals(finalEvolvedMetadata.schemas().size(), 3); | ||
| ops.commit(metadata, finalEvolvedMetadata); | ||
| TableMetadata result = ops.current(); | ||
| Assertions.assertEquals(3, result.schemas().size()); | ||
| Assertions.assertTrue(result.schema().sameSchema(finalEvolvedSchema)); | ||
| } finally { | ||
| if (spark != null) { | ||
| spark.sql("DROP TABLE openhouse.multiSchemaTest.t1"); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| void testMultiSchemaEvolutionColumnOrderingOnCreate() throws Exception { | ||
| SparkSession spark = null; | ||
| try { | ||
| spark = getSparkSession(); | ||
| TableIdentifier tableIdentifier = TableIdentifier.of("multiSchemaTest", "t2"); | ||
| OpenHouseCatalog ohCatalog = (OpenHouseCatalog) getOpenHouseCatalog(spark); | ||
| Schema schemaColumnOrdering = | ||
| new Schema( | ||
| Types.NestedField.optional(2, "name", Types.StringType.get()), | ||
| Types.NestedField.optional(1, "id", Types.IntegerType.get()), | ||
| Types.NestedField.optional(4, "newCol1", Types.IntegerType.get()), | ||
| Types.NestedField.optional(3, "newCol2", Types.IntegerType.get())); | ||
| Map<String, String> tableProperties = new HashMap<>(); | ||
| tableProperties.put("openhouse.tableType", "REPLICA_TABLE"); | ||
| tableProperties.put("openhouse.isTableReplicated", "true"); | ||
| tableProperties.put("client.table.schema", SchemaParser.toJson(schemaColumnOrdering)); | ||
| ohCatalog.createTable(tableIdentifier, schemaColumnOrdering, null, tableProperties); | ||
| TableOperations ops = ohCatalog.newTableOps(tableIdentifier); | ||
| TableMetadata metadata = ops.current(); | ||
| Assertions.assertEquals(metadata.schema().findColumnName(2), "name"); | ||
| Assertions.assertTrue(metadata.schema().sameSchema(schemaColumnOrdering)); | ||
| Schema schemaColumnOrdering2 = | ||
| new Schema( | ||
| Types.NestedField.optional(2, "name", Types.StringType.get()), | ||
| Types.NestedField.optional(1, "id", Types.IntegerType.get()), | ||
| Types.NestedField.optional(4, "newCol1", Types.IntegerType.get()), | ||
| Types.NestedField.optional(3, "newCol2", Types.IntegerType.get()), | ||
| Types.NestedField.optional(5, "newCol3", Types.IntegerType.get())); | ||
|
|
||
| Schema schemaColumnOrdering3 = | ||
| new Schema( | ||
| Types.NestedField.optional(2, "name", Types.StringType.get()), | ||
| Types.NestedField.optional(1, "id", Types.IntegerType.get()), | ||
| Types.NestedField.optional(4, "newCol1", Types.IntegerType.get()), | ||
| Types.NestedField.optional(3, "newCol2", Types.IntegerType.get()), | ||
| Types.NestedField.optional(5, "newCol3", Types.IntegerType.get()), | ||
| Types.NestedField.optional(6, "newCol4", Types.IntegerType.get()), | ||
| Types.NestedField.optional(7, "newCol5", Types.IntegerType.get())); | ||
|
|
||
| TableMetadata evolvedMetadata = | ||
| TableMetadata.buildFrom(metadata) | ||
| .addSchema(schemaColumnOrdering2, schemaColumnOrdering2.highestFieldId()) | ||
| .build(); | ||
| TableMetadata finalEvolvedMetadata = | ||
| TableMetadata.buildFrom(evolvedMetadata) | ||
| .addSchema(schemaColumnOrdering3, schemaColumnOrdering3.highestFieldId()) | ||
| .setCurrentSchema(2) | ||
| .build(); | ||
|
|
||
| Assertions.assertEquals(finalEvolvedMetadata.schemas().size(), 3); | ||
| ops.commit(metadata, finalEvolvedMetadata); | ||
| TableMetadata result = ops.current(); | ||
| Assertions.assertEquals(3, result.schemas().size()); | ||
| // Validate ordering of columns persists on creation | ||
| Assertions.assertEquals(result.schema().findColumnName(2), "name"); | ||
| Assertions.assertTrue(result.schema().sameSchema(schemaColumnOrdering3)); | ||
| } finally { | ||
| if (spark != null) { | ||
| spark.sql("DROP TABLE openhouse.multiSchemaTest.t2"); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private Catalog getOpenHouseCatalog(SparkSession spark) { | ||
| final Map<String, String> catalogProperties = new HashMap<>(); | ||
| final String catalogPropertyPrefix = "spark.sql.catalog.openhouse."; | ||
| final Map<String, String> sparkProperties = JavaConverters.mapAsJavaMap(spark.conf().getAll()); | ||
| for (Map.Entry<String, String> entry : sparkProperties.entrySet()) { | ||
| if (entry.getKey().startsWith(catalogPropertyPrefix)) { | ||
| catalogProperties.put( | ||
| entry.getKey().substring(catalogPropertyPrefix.length()), entry.getValue()); | ||
| } | ||
| } | ||
| // this initializes the catalog based on runtime Catalog class passed in catalog-impl conf. | ||
| return CatalogUtil.loadCatalog( | ||
| sparkProperties.get("spark.sql.catalog.openhouse.catalog-impl"), | ||
| "openhouse", | ||
| catalogProperties, | ||
| spark.sparkContext().hadoopConfiguration()); | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.