1
0

[HUDI-3838] Implemented drop partition column feature for delta streamer code path (#5294)

* [HUDI-3838] Implemented drop partition column feature for delta streamer code path

* Ensure drop partition table config is updated in hoodie.props

Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
Vinoth Govindarajan
2022-04-12 05:40:30 -07:00
committed by GitHub
parent 101b82a679
commit d16740976e
3 changed files with 70 additions and 2 deletions

View File

@@ -441,6 +441,17 @@ public class HoodieAvroUtils {
return records.stream().map(r -> rewriteRecord(r, newSchema)).collect(Collectors.toList());
}
/**
* Given an Avro record and list of columns to remove, this method removes the list of columns from
* the given avro record using rewriteRecord method.
* <p>
* To better understand how it removes please check {@link #rewriteRecord(GenericRecord, Schema)}
*/
public static GenericRecord removeFields(GenericRecord record, List<String> fieldsToRemove) {
Schema newSchema = removeFields(record.getSchema(), fieldsToRemove);
return rewriteRecord(record, newSchema);
}
private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) {
Schema oldSchema = oldRecord.getSchema();
Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name());

View File

@@ -32,6 +32,7 @@ import java.io.IOException;
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
@@ -227,6 +228,35 @@ public class TestHoodieAvroUtils {
assertEquals(NUM_FIELDS_IN_EXAMPLE_SCHEMA, schemaWithoutMetaCols.getFields().size());
}
@Test
public void testRemoveFields() {
// partitioned table test.
String schemaStr = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ "
+ "{\"name\": \"timestamp\",\"type\": \"double\"},{\"name\": \"_row_key\", \"type\": \"string\"},"
+ "{\"name\": \"non_pii_col\", \"type\": \"string\"}]},";
Schema expectedSchema = new Schema.Parser().parse(schemaStr);
GenericRecord rec = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA));
rec.put("_row_key", "key1");
rec.put("non_pii_col", "val1");
rec.put("pii_col", "val2");
rec.put("timestamp", 3.5);
GenericRecord rec1 = HoodieAvroUtils.removeFields(rec, Arrays.asList("pii_col"));
assertEquals("key1", rec1.get("_row_key"));
assertEquals("val1", rec1.get("non_pii_col"));
assertEquals(3.5, rec1.get("timestamp"));
assertNull(rec1.get("pii_col"));
assertEquals(expectedSchema, rec1.getSchema());
// non-partitioned table test with empty list of fields.
schemaStr = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ "
+ "{\"name\": \"timestamp\",\"type\": \"double\"},{\"name\": \"_row_key\", \"type\": \"string\"},"
+ "{\"name\": \"non_pii_col\", \"type\": \"string\"},"
+ "{\"name\": \"pii_col\", \"type\": \"string\"}]},";
expectedSchema = new Schema.Parser().parse(schemaStr);
rec1 = HoodieAvroUtils.removeFields(rec, Arrays.asList(""));
assertEquals(expectedSchema, rec1.getSchema());
}
@Test
public void testGetNestedFieldVal() {
GenericRecord rec = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA));