-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-5646] Guard dropping columns by a config, do not allow by default #7787
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
8837748
e80376b
1faf54c
fa4a383
937958f
396de18
b9ac6f2
9e0445c
0db5034
41584f7
0962153
fe056bb
f75bc67
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 |
|---|---|---|
|
|
@@ -18,8 +18,6 @@ | |
|
|
||
| package org.apache.hudi.client; | ||
|
|
||
| import org.apache.avro.Schema; | ||
| import org.apache.avro.generic.GenericRecord; | ||
| import org.apache.hudi.avro.AvroSchemaUtils; | ||
| import org.apache.hudi.avro.HoodieAvroUtils; | ||
| import org.apache.hudi.common.model.HoodieAvroRecord; | ||
|
|
@@ -34,10 +32,17 @@ | |
| import org.apache.hudi.config.HoodieCompactionConfig; | ||
| import org.apache.hudi.config.HoodieIndexConfig; | ||
| import org.apache.hudi.config.HoodieWriteConfig; | ||
| import org.apache.hudi.exception.HoodieInsertException; | ||
| import org.apache.hudi.exception.HoodieUpsertException; | ||
| import org.apache.hudi.index.HoodieIndex.IndexType; | ||
| import org.apache.hudi.testutils.HoodieClientTestBase; | ||
| import org.apache.hudi.testutils.HoodieClientTestUtils; | ||
|
|
||
| import org.apache.avro.Schema; | ||
| import org.apache.avro.generic.GenericRecord; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.junit.jupiter.params.ParameterizedTest; | ||
| import org.junit.jupiter.params.provider.ValueSource; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.List; | ||
|
|
@@ -79,36 +84,35 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { | |
|
|
||
| @Test | ||
| public void testSchemaCompatibilityBasic() { | ||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA), | ||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA, false), | ||
| "Same schema is compatible"); | ||
|
|
||
| String reorderedSchema = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + TIP_NESTED_SCHEMA + FARE_NESTED_SCHEMA | ||
| + MAP_TYPE_SCHEMA + TRIP_SCHEMA_SUFFIX; | ||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, reorderedSchema), | ||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, reorderedSchema, false), | ||
| "Reordered fields are compatible"); | ||
| assertTrue(isSchemaCompatible(reorderedSchema, TRIP_EXAMPLE_SCHEMA), | ||
| assertTrue(isSchemaCompatible(reorderedSchema, TRIP_EXAMPLE_SCHEMA, false), | ||
| "Reordered fields are compatible"); | ||
|
|
||
| String renamedSchema = TRIP_EXAMPLE_SCHEMA.replace("tip_history", "tip_future"); | ||
|
|
||
| // NOTE: That even though renames could be carried over as "column drop" and "column add" | ||
| // both of which are legitimate operations, no data carry-over will occur (exactly b/c | ||
| // it's an old column being dropped, and the new one being added) | ||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedSchema), | ||
| assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedSchema, false), | ||
| "Renaming fields is essentially: dropping old field, created a new one"); | ||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedSchema, true), | ||
| "Renaming fields is essentially: dropping old field, created a new one"); | ||
|
|
||
| String renamedRecordSchema = TRIP_EXAMPLE_SCHEMA.replace("triprec", "triprec_renamed"); | ||
| assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedRecordSchema), | ||
| assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedRecordSchema, false), | ||
| "Renamed record name is not compatible"); | ||
|
|
||
| String swappedFieldSchema = TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA.replace("city_to_state", "fare") | ||
| + FARE_NESTED_SCHEMA.replace("fare", "city_to_state") + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX; | ||
| assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, swappedFieldSchema), | ||
| assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, swappedFieldSchema, false), | ||
| "Swapped fields are not compatible"); | ||
|
|
||
| String typeChangeSchemaDisallowed = TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA | ||
| + TIP_NESTED_SCHEMA.replace("string", "boolean") + TRIP_SCHEMA_SUFFIX; | ||
| assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, typeChangeSchemaDisallowed), | ||
| assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, typeChangeSchemaDisallowed, false), | ||
| "Incompatible field type change is not allowed"); | ||
|
|
||
| // Array of allowed schema field type transitions | ||
|
|
@@ -119,43 +123,42 @@ public void testSchemaCompatibilityBasic() { | |
| for (String[] fieldChange : allowedFieldChanges) { | ||
| String fromSchema = TRIP_SCHEMA_PREFIX + EXTRA_FIELD_SCHEMA.replace("string", fieldChange[0]) + TRIP_SCHEMA_SUFFIX; | ||
| String toSchema = TRIP_SCHEMA_PREFIX + EXTRA_FIELD_SCHEMA.replace("string", fieldChange[1]) + TRIP_SCHEMA_SUFFIX; | ||
| assertTrue(isSchemaCompatible(fromSchema, toSchema), | ||
| assertTrue(isSchemaCompatible(fromSchema, toSchema, false), | ||
| "Compatible field type change is not allowed"); | ||
| if (!fieldChange[0].equals("byte") && fieldChange[1].equals("byte")) { | ||
| assertFalse(isSchemaCompatible(toSchema, fromSchema), | ||
| assertFalse(isSchemaCompatible(toSchema, fromSchema, false), | ||
| "Incompatible field type change is allowed"); | ||
| } | ||
| } | ||
|
|
||
| // Names and aliases should match | ||
| String fromSchema = TRIP_SCHEMA_PREFIX + EXTRA_FIELD_SCHEMA + TRIP_SCHEMA_SUFFIX; | ||
| String toSchema = TRIP_SCHEMA_PREFIX.replace("triprec", "new_triprec") + EXTRA_FIELD_SCHEMA + TRIP_SCHEMA_SUFFIX; | ||
| assertFalse(isSchemaCompatible(fromSchema, toSchema), "Field names should match"); | ||
| assertFalse(isSchemaCompatible(toSchema, fromSchema), "Field names should match"); | ||
| assertFalse(isSchemaCompatible(fromSchema, toSchema, false), "Field names should match"); | ||
| assertFalse(isSchemaCompatible(toSchema, fromSchema, false), "Field names should match"); | ||
|
|
||
|
|
||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED), | ||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED, false), | ||
| "Added field with default is compatible (Evolved Schema)"); | ||
|
|
||
| String multipleAddedFieldSchema = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA | ||
| + TIP_NESTED_SCHEMA + EXTRA_FIELD_SCHEMA + EXTRA_FIELD_SCHEMA.replace("new_field", "new_new_field") | ||
| + TRIP_SCHEMA_SUFFIX; | ||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, multipleAddedFieldSchema), | ||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, multipleAddedFieldSchema, false), | ||
| "Multiple added fields with defaults are compatible"); | ||
|
|
||
| assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, | ||
| TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA | ||
| + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + EXTRA_FIELD_WITHOUT_DEFAULT_SCHEMA + TRIP_SCHEMA_SUFFIX), | ||
| assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA | ||
| + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + EXTRA_FIELD_WITHOUT_DEFAULT_SCHEMA + TRIP_SCHEMA_SUFFIX, false), | ||
| "Added field without default and not nullable is not compatible (Evolved Schema)"); | ||
|
|
||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, | ||
| TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA | ||
| + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX + EXTRA_FIELD_NULLABLE_SCHEMA), | ||
| assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA | ||
| + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX + EXTRA_FIELD_NULLABLE_SCHEMA, false), | ||
| "Added nullable field is compatible (Evolved Schema)"); | ||
| } | ||
|
|
||
| @Test | ||
| public void testMORTable() throws Exception { | ||
| @ParameterizedTest | ||
| @ValueSource(booleans = {false, true}) | ||
| public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { | ||
| tableType = HoodieTableType.MERGE_ON_READ; | ||
|
|
||
| // Create the table | ||
|
|
@@ -165,7 +168,7 @@ public void testMORTable() throws Exception { | |
| .setTimelineLayoutVersion(VERSION_1) | ||
| .initTable(metaClient.getHadoopConf(), metaClient.getBasePath()); | ||
|
|
||
| HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA); | ||
| HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA, shouldAllowDroppedColumns); | ||
| SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); | ||
|
|
||
| // Initial inserts with TRIP_EXAMPLE_SCHEMA | ||
|
|
@@ -194,20 +197,26 @@ public void testMORTable() throws Exception { | |
| checkReadRecords("000", numRecords); | ||
|
|
||
| // Insert with evolved schema (column dropped) is allowed | ||
| HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED); | ||
| HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED, shouldAllowDroppedColumns); | ||
| client = getHoodieWriteClient(hoodieDevolvedWriteConfig); | ||
| final List<HoodieRecord> failedRecords = generateInsertsWithSchema("005", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED); | ||
| // We cannot use insertBatch directly here because we want to insert records | ||
| // with a evolved schema and insertBatch inserts records using the TRIP_EXAMPLE_SCHEMA. | ||
| writeBatch(client, "005", "004", Option.empty(), "003", numRecords, | ||
| (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, numRecords, 2 * numRecords, 5, false); | ||
| try { | ||
| writeBatch(client, "005", "004", Option.empty(), "003", numRecords, | ||
| (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, numRecords, 2 * numRecords, 5, false); | ||
| assertTrue(shouldAllowDroppedColumns); | ||
| } catch (HoodieInsertException e) { | ||
| assertFalse(shouldAllowDroppedColumns); | ||
| return; | ||
| } | ||
|
|
||
| // Update with evolved schema (column dropped) is allowed | ||
| // Update with evolved schema (column dropped) might be allowed depending on config set. | ||
| updateBatch(hoodieDevolvedWriteConfig, client, "006", "005", Option.empty(), | ||
| initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, numUpdateRecords, 2 * numRecords, 0); | ||
|
|
||
| // Insert with an evolved scheme is allowed | ||
| HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED); | ||
| HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED, shouldAllowDroppedColumns); | ||
| client = getHoodieWriteClient(hoodieEvolvedWriteConfig); | ||
|
|
||
| // We cannot use insertBatch directly here because we want to insert records | ||
|
|
@@ -230,19 +239,28 @@ public void testMORTable() throws Exception { | |
|
|
||
| // Now try updating w/ the original schema (should succeed) | ||
| client = getHoodieWriteClient(hoodieWriteConfig); | ||
| updateBatch(hoodieWriteConfig, client, "009", "008", Option.empty(), | ||
| initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, numUpdateRecords, 4 * numRecords, 9); | ||
| try { | ||
| updateBatch(hoodieWriteConfig, client, "009", "008", Option.empty(), | ||
| initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, numUpdateRecords, 4 * numRecords, 9); | ||
| assertTrue(shouldAllowDroppedColumns); | ||
| } catch (HoodieUpsertException e) { | ||
|
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. similar comment as above |
||
| assertFalse(shouldAllowDroppedColumns); | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| public void testCopyOnWriteTable() throws Exception { | ||
| @ParameterizedTest | ||
| @ValueSource(booleans = {false, true}) | ||
| public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Exception { | ||
| // Create the table | ||
| HoodieTableMetaClient.withPropertyBuilder() | ||
| .fromMetaClient(metaClient) | ||
| .setTimelineLayoutVersion(VERSION_1) | ||
| .initTable(metaClient.getHadoopConf(), metaClient.getBasePath()); | ||
|
|
||
| HoodieWriteConfig hoodieWriteConfig = getWriteConfigBuilder(TRIP_EXAMPLE_SCHEMA).withRollbackUsingMarkers(false).build(); | ||
| HoodieWriteConfig hoodieWriteConfig = getWriteConfigBuilder(TRIP_EXAMPLE_SCHEMA) | ||
| .withRollbackUsingMarkers(false) | ||
| .withAllowAutoEvolutionColumnDrop(shouldAllowDroppedColumns) | ||
| .build(); | ||
| SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); | ||
|
|
||
| // Initial inserts with TRIP_EXAMPLE_SCHEMA | ||
|
|
@@ -266,19 +284,25 @@ public void testCopyOnWriteTable() throws Exception { | |
| checkReadRecords("000", numRecords); | ||
|
|
||
| // Inserting records w/ new evolved schema (w/ tip column dropped) | ||
| HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED); | ||
| HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED, shouldAllowDroppedColumns); | ||
| client = getHoodieWriteClient(hoodieDevolvedWriteConfig); | ||
| final List<HoodieRecord> failedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED); | ||
| writeBatch(client, "004", "003", Option.empty(), "003", numRecords, | ||
| (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords * 2, 1, false); | ||
| try { | ||
| writeBatch(client, "004", "003", Option.empty(), "003", numRecords, | ||
| (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords * 2, 1, false); | ||
| assertTrue(shouldAllowDroppedColumns); | ||
| } catch (HoodieInsertException e) { | ||
| assertFalse(shouldAllowDroppedColumns); | ||
|
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. ditto |
||
| return; | ||
| } | ||
|
|
||
| // Updating records w/ new evolved schema | ||
| updateBatch(hoodieDevolvedWriteConfig, client, "005", "004", Option.empty(), | ||
| initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, | ||
| numUpdateRecords, 2 * numRecords, 5); | ||
|
|
||
| // Inserting with evolved schema is allowed | ||
| HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED); | ||
| HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED, shouldAllowDroppedColumns); | ||
| client = getHoodieWriteClient(hoodieEvolvedWriteConfig); | ||
| final List<HoodieRecord> evolvedRecords = generateInsertsWithSchema("006", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED); | ||
| // We cannot use insertBatch directly here because we want to insert records | ||
|
|
@@ -299,9 +323,14 @@ public void testCopyOnWriteTable() throws Exception { | |
|
|
||
| // Now try updating w/ the original schema (should succeed) | ||
| client = getHoodieWriteClient(hoodieWriteConfig); | ||
| updateBatch(hoodieWriteConfig, client, "008", "007", Option.empty(), | ||
| initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, | ||
| numUpdateRecords, 3 * numRecords, 8); | ||
| try { | ||
| updateBatch(hoodieWriteConfig, client, "008", "007", Option.empty(), | ||
| initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, | ||
| numUpdateRecords, 3 * numRecords, 8); | ||
| assertTrue(shouldAllowDroppedColumns); | ||
| } catch (HoodieUpsertException e) { | ||
| assertFalse(shouldAllowDroppedColumns); | ||
|
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. ditto |
||
| } | ||
| } | ||
|
|
||
| private void checkReadRecords(String instantTime, int numExpectedRecords) throws IOException { | ||
|
|
@@ -362,8 +391,8 @@ private List<HoodieRecord> convertToSchema(List<HoodieRecord> records, String sc | |
| }).collect(Collectors.toList()); | ||
| } | ||
|
|
||
| private HoodieWriteConfig getWriteConfig(String schema) { | ||
| return getWriteConfigBuilder(schema).build(); | ||
| private HoodieWriteConfig getWriteConfig(String schema, boolean shouldAllowDroppedColumns) { | ||
| return getWriteConfigBuilder(schema).withAllowAutoEvolutionColumnDrop(shouldAllowDroppedColumns).build(); | ||
| } | ||
|
|
||
| private HoodieWriteConfig.Builder getWriteConfigBuilder(String schema) { | ||
|
|
@@ -373,8 +402,8 @@ private HoodieWriteConfig.Builder getWriteConfigBuilder(String schema) { | |
| .withAvroSchemaValidate(true); | ||
| } | ||
|
|
||
| private static boolean isSchemaCompatible(String oldSchema, String newSchema) { | ||
| return AvroSchemaUtils.isSchemaCompatible(new Schema.Parser().parse(oldSchema), new Schema.Parser().parse(newSchema)); | ||
| private static boolean isSchemaCompatible(String oldSchema, String newSchema, boolean shouldAllowDroppedColumns) { | ||
| return AvroSchemaUtils.isSchemaCompatible(new Schema.Parser().parse(oldSchema), new Schema.Parser().parse(newSchema), shouldAllowDroppedColumns); | ||
| } | ||
|
|
||
| @Override | ||
|
|
||
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.
after L209, we should also do, before catch block