the-other-tim-brown commented on code in PR #17604: URL: https://github.com/apache/hudi/pull/17604#discussion_r2728396041
########## hudi-common/src/main/java/org/apache/hudi/common/table/read/PartialMergerWithKeepValues.java: ########## @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.read; + +import org.apache.hudi.common.engine.RecordContext; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; +import org.apache.hudi.common.util.VisibleForTesting; +import org.apache.hudi.common.util.collection.Pair; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Class to assist with merging two versions of the record that may contain partial updates using + * {@link org.apache.hudi.common.table.PartialUpdateMode#KEEP_VALUES} mode. + */ +public class PartialMergerWithKeepValues<T> implements Serializable { + private final Map<HoodieSchema, Map<String, Integer>> + fieldNameToIdMappingCache = new ConcurrentHashMap<>(); + private final Map<Pair<Pair<HoodieSchema, HoodieSchema>, HoodieSchema>, HoodieSchema> + mergedSchemaCache = new ConcurrentHashMap<>(); + + /** + * Merges records which can contain partial updates. + * + * @param lowOrderRecord record with lower commit time or lower ordering value + * @param lowOrderSchema The schema of the older record + * @param highOrderRecord record with higher commit time or higher ordering value + * @param highOrderSchema The schema of highOrderRecord + * @param newSchema The schema of the new incoming record + * @return The merged record and schema. + */ + Pair<BufferedRecord<T>, HoodieSchema> mergePartialRecords(BufferedRecord<T> lowOrderRecord, + HoodieSchema lowOrderSchema, + BufferedRecord<T> highOrderRecord, + HoodieSchema highOrderSchema, + HoodieSchema newSchema, + RecordContext<T> recordContext) { + // The merged schema contains fields that only appear in either older and/or newer record. + HoodieSchema mergedSchema = + getCachedMergedSchema(lowOrderSchema, highOrderSchema, newSchema); + boolean isNewerPartial = isPartial(highOrderSchema, mergedSchema); + if (!isNewerPartial) { + return Pair.of(highOrderRecord, highOrderSchema); + } + Set<String> fieldNamesInNewRecord = + getCachedFieldNameToIdMapping(highOrderSchema).keySet(); + // Collect field values. + List<HoodieSchemaField> fields = mergedSchema.getFields(); + Object[] fieldVals = new Object[fields.size()]; + int idx = 0; + List<HoodieSchemaField> mergedSchemaFields = mergedSchema.getFields(); + for (HoodieSchemaField mergedSchemaField : mergedSchemaFields) { + String fieldName = mergedSchemaField.name(); + if (fieldNamesInNewRecord.contains(fieldName)) { // field present in newer record. + fieldVals[idx++] = recordContext.getValue(highOrderRecord.getRecord(), highOrderSchema, fieldName); + } else { // if not present in newer record pick from old record + fieldVals[idx++] = recordContext.getValue(lowOrderRecord.getRecord(), lowOrderSchema, fieldName); + } + } + // Build merged record. + T engineRecord = recordContext.constructEngineRecord(mergedSchema, fieldVals); + BufferedRecord<T> mergedRecord = new BufferedRecord<>( + highOrderRecord.getRecordKey(), + highOrderRecord.getOrderingValue(), + engineRecord, + recordContext.encodeSchema(mergedSchema), + highOrderRecord.getHoodieOperation()); + return Pair.of(mergedRecord, mergedSchema); + } + + /** + * @param hoodieSchema Hoodie schema. + * @return The field name to ID mapping. + */ + Map<String, Integer> getCachedFieldNameToIdMapping(HoodieSchema hoodieSchema) { + return fieldNameToIdMappingCache.computeIfAbsent(hoodieSchema, schema -> { + Map<String, Integer> schemaFieldIdMapping = new HashMap<>(); + int fieldId = 0; + for (HoodieSchemaField field : schema.getFields()) { + schemaFieldIdMapping.put(field.name(), fieldId); + fieldId++; + } + return schemaFieldIdMapping; + }); + } + + /** + * Merges the two schemas so the merged schema contains all the fields from the two schemas, + * with the same ordering of fields based on the provided reader schema. + * + * @param oldSchema Old schema. + * @param newSchema New schema. + * @param readerSchema Reader schema containing all the fields to read. + * @return The merged Avro schema. + */ + HoodieSchema getCachedMergedSchema(HoodieSchema oldSchema, + HoodieSchema newSchema, + HoodieSchema readerSchema) { + return mergedSchemaCache.computeIfAbsent( + Pair.of(Pair.of(oldSchema, newSchema), readerSchema), schemaPair -> { + HoodieSchema schema1 = schemaPair.getLeft().getLeft(); + HoodieSchema schema2 = schemaPair.getLeft().getRight(); + HoodieSchema refSchema = schemaPair.getRight(); + Set<String> schema1Keys = + getCachedFieldNameToIdMapping(schema1).keySet(); + Set<String> schema2Keys = + getCachedFieldNameToIdMapping(schema2).keySet(); + List<HoodieSchemaField> mergedFieldList = new ArrayList<>(); + for (int i = 0; i < refSchema.getFields().size(); i++) { + HoodieSchemaField field = refSchema.getFields().get(i); + if (schema1Keys.contains(field.name()) || schema2Keys.contains(field.name())) { + mergedFieldList.add(HoodieSchemaField.of( + field.name(), + field.schema(), + field.doc().orElse(null), + field.defaultVal().orElse(null))); + } + } + HoodieSchema mergedSchema = new HoodieSchema.Builder(HoodieSchemaType.RECORD) + .setName(readerSchema.getName()) + .setDoc(readerSchema.getDoc().orElse(null)) + .setNamespace(readerSchema.getNamespace().orElse(null)) + .setFields(mergedFieldList) + .build(); + return mergedSchema; + }); + } + + /** + * @param schema Avro schema to check. + * @param mergedSchema The merged schema for the merged record. + * @return whether the Avro schema is partial compared to the merged schema. + */ + @VisibleForTesting + public static boolean isPartial(HoodieSchema schema, HoodieSchema mergedSchema) { Review Comment: Make this package private if it is only visible for testing ########## hudi-common/src/test/java/org/apache/hudi/common/table/read/TestPartialMergerWithKeepValues.java: ########## @@ -0,0 +1,349 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.read; + +import org.apache.hudi.common.engine.RecordContext; +import org.apache.hudi.common.model.HoodieOperation; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class TestPartialMergerWithKeepValues { + private PartialMergerWithKeepValues<IndexedRecord> keepValuesPartialMergingUtils; + private RecordContext<IndexedRecord> mockRecordContext; + private HoodieSchema fullSchema; + private HoodieSchema partialSchema; + private HoodieSchema readerSchema; + private Schema avroFullSchema; + private Schema avroPartialSchema; + private Schema avroReaderSchema; + + @BeforeEach + void setUp() { + keepValuesPartialMergingUtils = new PartialMergerWithKeepValues<>(); + mockRecordContext = mock(RecordContext.class); + + // Create Avro schemas first + avroFullSchema = Schema.createRecord("TestRecord", "Test record", "test", false); Review Comment: Just directly construct HoodieSchema's and then use `toAvroSchema` where required ########## hudi-common/src/main/java/org/apache/hudi/common/table/read/PartialMergerWithKeepValues.java: ########## @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.read; + +import org.apache.hudi.common.engine.RecordContext; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; +import org.apache.hudi.common.util.VisibleForTesting; +import org.apache.hudi.common.util.collection.Pair; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Class to assist with merging two versions of the record that may contain partial updates using + * {@link org.apache.hudi.common.table.PartialUpdateMode#KEEP_VALUES} mode. + */ +public class PartialMergerWithKeepValues<T> implements Serializable { + private final Map<HoodieSchema, Map<String, Integer>> + fieldNameToIdMappingCache = new ConcurrentHashMap<>(); + private final Map<Pair<Pair<HoodieSchema, HoodieSchema>, HoodieSchema>, HoodieSchema> + mergedSchemaCache = new ConcurrentHashMap<>(); Review Comment: The merger is not used across threads so you can use a HashMap which will have better performance ########## hudi-common/src/main/java/org/apache/hudi/common/table/read/PartialUpdateHandler.java: ########## @@ -82,17 +85,43 @@ BufferedRecord<T> partialMerge(BufferedRecord<T> highOrderRecord, } switch (partialUpdateMode) { + case KEEP_VALUES: + return reconcileBasedOnKeepValues(highOrderRecord, lowOrderRecord, highOrderSchema, lowOrderSchema, newSchema); case IGNORE_DEFAULTS: return reconcileDefaultValues( highOrderRecord, lowOrderRecord, highOrderSchema, lowOrderSchema, newSchema); case FILL_UNAVAILABLE: return reconcileMarkerValues( highOrderRecord, lowOrderRecord, highOrderSchema, lowOrderSchema, newSchema); default: - return highOrderRecord; + throw new HoodieIOException("Unsupported PartialUpdateMode " + partialUpdateMode + " detected"); } } + /** + * Reconcile two versions of the record based on KEEP_VALUES. + * i.e for values missing from new record, we pick from older record, if not, value from new record is picked for each column. + * @param highOrderRecord record with higher commit time or higher ordering value + * @param lowOrderRecord record with lower commit time or lower ordering value + * @param highOrderSchema The schema of highOrderRecord + * @param lowOrderSchema The schema of the older record + * @param newSchema The schema of the new incoming record + * @return the merged record of type {@link BufferedRecord} + */ + BufferedRecord<T> reconcileBasedOnKeepValues(BufferedRecord<T> highOrderRecord, + BufferedRecord<T> lowOrderRecord, + HoodieSchema highOrderSchema, + HoodieSchema lowOrderSchema, + HoodieSchema newSchema) { + return (BufferedRecord<T>) keepValuesPartialMergingUtils.mergePartialRecords( Review Comment: nit: the casting is not required ########## hudi-common/src/test/java/org/apache/hudi/common/table/read/TestPartialMergerWithKeepValues.java: ########## @@ -0,0 +1,349 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.read; + +import org.apache.hudi.common.engine.RecordContext; +import org.apache.hudi.common.model.HoodieOperation; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class TestPartialMergerWithKeepValues { + private PartialMergerWithKeepValues<IndexedRecord> keepValuesPartialMergingUtils; + private RecordContext<IndexedRecord> mockRecordContext; + private HoodieSchema fullSchema; + private HoodieSchema partialSchema; + private HoodieSchema readerSchema; + private Schema avroFullSchema; + private Schema avroPartialSchema; + private Schema avroReaderSchema; + + @BeforeEach + void setUp() { + keepValuesPartialMergingUtils = new PartialMergerWithKeepValues<>(); + mockRecordContext = mock(RecordContext.class); + + // Create Avro schemas first + avroFullSchema = Schema.createRecord("TestRecord", "Test record", "test", false); + avroFullSchema.setFields(Arrays.asList( + new Schema.Field("id", Schema.create(Schema.Type.STRING), "ID field", null), + new Schema.Field("name", Schema.create(Schema.Type.STRING), "Name field", null), + new Schema.Field("age", Schema.create(Schema.Type.INT), "Age field", null), + new Schema.Field("city", Schema.create(Schema.Type.STRING), "City field", null) + )); + + avroPartialSchema = Schema.createRecord("TestRecord", "Test record", "test", false); + avroPartialSchema.setFields(Arrays.asList( + new Schema.Field("id", Schema.create(Schema.Type.STRING), "ID field", null), + new Schema.Field("name", Schema.create(Schema.Type.STRING), "Name field", null) + )); + + avroReaderSchema = Schema.createRecord("TestRecord", "Test record", "test", false); + avroReaderSchema.setFields(Arrays.asList( + new Schema.Field("id", Schema.create(Schema.Type.STRING), "ID field", null), + new Schema.Field("name", Schema.create(Schema.Type.STRING), "Name field", null), + new Schema.Field("age", Schema.create(Schema.Type.INT), "Age field", null), + new Schema.Field("city", Schema.create(Schema.Type.STRING), "City field", null) + )); + + // Wrap with HoodieSchema + fullSchema = HoodieSchema.fromAvroSchema(avroFullSchema); + partialSchema = HoodieSchema.fromAvroSchema(avroPartialSchema); + readerSchema = HoodieSchema.fromAvroSchema(avroReaderSchema); + } + + @Test + void testGetCachedFieldNameToIdMapping() { + Map<String, Integer> fieldNameToIdMapping = keepValuesPartialMergingUtils.getCachedFieldNameToIdMapping(fullSchema); + + assertNotNull(fieldNameToIdMapping); + assertEquals(4, fieldNameToIdMapping.size()); + assertEquals(0, fieldNameToIdMapping.get("id")); + assertEquals(1, fieldNameToIdMapping.get("name")); + assertEquals(2, fieldNameToIdMapping.get("age")); + assertEquals(3, fieldNameToIdMapping.get("city")); + + // Test caching - should return the same map for the same schema + Map<String, Integer> cachedMapping = keepValuesPartialMergingUtils.getCachedFieldNameToIdMapping(fullSchema); + assertSame(fieldNameToIdMapping, cachedMapping); + } + + @Test + void testGetCachedMergedSchema() { + HoodieSchema mergedSchema = keepValuesPartialMergingUtils.getCachedMergedSchema(partialSchema, fullSchema, readerSchema); + + assertNotNull(mergedSchema); + assertEquals(4, mergedSchema.getFields().size()); + assertEquals("id", mergedSchema.getFields().get(0).name()); + assertEquals("name", mergedSchema.getFields().get(1).name()); + assertEquals("age", mergedSchema.getFields().get(2).name()); + assertEquals("city", mergedSchema.getFields().get(3).name()); + + // Test caching - should return the same schema for the same input + HoodieSchema cachedSchema = keepValuesPartialMergingUtils.getCachedMergedSchema(partialSchema, fullSchema, readerSchema); + assertSame(mergedSchema, cachedSchema); + } + + @Test + void testGetCachedMergedSchemaWithDifferentFieldOrder() { + // Create a schema with different field order + Schema avroReorderedSchema = Schema.createRecord("TestRecord", "Test record", "test", false); + avroReorderedSchema.setFields(Arrays.asList( + new Schema.Field("age", Schema.create(Schema.Type.INT), "Age field", null), + new Schema.Field("city", Schema.create(Schema.Type.STRING), "City field", null), + new Schema.Field("id", Schema.create(Schema.Type.STRING), "ID field", null), + new Schema.Field("name", Schema.create(Schema.Type.STRING), "Name field", null) + )); + HoodieSchema reorderedSchema = HoodieSchema.fromAvroSchema(avroReorderedSchema); + + HoodieSchema mergedSchema = keepValuesPartialMergingUtils.getCachedMergedSchema(partialSchema, reorderedSchema, readerSchema); + + // Should follow the reader schema order + assertEquals(4, mergedSchema.getFields().size()); + assertEquals("id", mergedSchema.getFields().get(0).name()); + assertEquals("name", mergedSchema.getFields().get(1).name()); + assertEquals("age", mergedSchema.getFields().get(2).name()); + assertEquals("city", mergedSchema.getFields().get(3).name()); + } + + @Test + void testIsPartial() { + // Test when schema is partial compared to merged schema + assertTrue(PartialMergerWithKeepValues.isPartial(partialSchema, fullSchema)); + + // Test when schema is not partial (same as merged schema) + assertFalse(PartialMergerWithKeepValues.isPartial(fullSchema, fullSchema)); + + // Test when schema has more fields than merged schema + Schema avroExtendedSchema = Schema.createRecord("TestRecord", "Test record", "test", false); + avroExtendedSchema.setFields(Arrays.asList( + new Schema.Field("id", Schema.create(Schema.Type.STRING), "ID field", null), + new Schema.Field("name", Schema.create(Schema.Type.STRING), "Name field", null), + new Schema.Field("age", Schema.create(Schema.Type.INT), "Age field", null), + new Schema.Field("city", Schema.create(Schema.Type.STRING), "City field", null), + new Schema.Field("extra", Schema.create(Schema.Type.STRING), "Extra field", null) + )); + HoodieSchema extendedSchema = HoodieSchema.fromAvroSchema(avroExtendedSchema); + assertTrue(PartialMergerWithKeepValues.isPartial(extendedSchema, fullSchema)); + } + + @Test + void testMergePartialRecordsWithNonPartialNewer() { + // Create test records + GenericRecord olderRecord = new GenericData.Record(avroFullSchema); + olderRecord.put("id", "1"); + olderRecord.put("name", "John"); + olderRecord.put("age", 25); + olderRecord.put("city", "New York"); + + GenericRecord newerRecord = new GenericData.Record(avroFullSchema); + newerRecord.put("id", "1"); + newerRecord.put("name", "Jane"); + newerRecord.put("age", 30); + newerRecord.put("city", "Boston"); + + BufferedRecord<IndexedRecord> older = new BufferedRecord<>("1", 1L, olderRecord, 1, null); + BufferedRecord<IndexedRecord> newer = new BufferedRecord<>("1", 2L, newerRecord, 1, null); + + // When newer schema is not partial, should return newer record as-is + Pair<BufferedRecord<IndexedRecord>, HoodieSchema> result = keepValuesPartialMergingUtils.mergePartialRecords( + older, fullSchema, newer, fullSchema, readerSchema, mockRecordContext); + + assertEquals(newer, result.getLeft()); + assertEquals(fullSchema, result.getRight()); + } + + @Test + void testMergePartialRecordsWithPartialNewer() { + // Create test records + GenericRecord olderRecord = new GenericData.Record(avroFullSchema); + olderRecord.put("id", "1"); + olderRecord.put("name", "John"); + olderRecord.put("age", 25); + olderRecord.put("city", "New York"); + + GenericRecord newerRecord = new GenericData.Record(avroPartialSchema); + newerRecord.put("id", "1"); + newerRecord.put("name", "Jane"); + + BufferedRecord<IndexedRecord> older = new BufferedRecord<>("1", 1L, olderRecord, 1, null); + BufferedRecord<IndexedRecord> newer = new BufferedRecord<>("1", 2L, newerRecord, 1, null); + + // Mock record context behavior + when(mockRecordContext.getValue(eq(olderRecord), eq(fullSchema), eq("id"))).thenReturn("1"); + when(mockRecordContext.getValue(eq(olderRecord), eq(fullSchema), eq("name"))).thenReturn("John"); + when(mockRecordContext.getValue(eq(olderRecord), eq(fullSchema), eq("age"))).thenReturn(25); + when(mockRecordContext.getValue(eq(olderRecord), eq(fullSchema), eq("city"))).thenReturn("New York"); + + when(mockRecordContext.getValue(eq(newerRecord), eq(partialSchema), eq("id"))).thenReturn("1"); + when(mockRecordContext.getValue(eq(newerRecord), eq(partialSchema), eq("name"))).thenReturn("Jane"); + + GenericRecord mergedRecord = new GenericData.Record(avroReaderSchema); + mergedRecord.put("id", "1"); + mergedRecord.put("name", "Jane"); + mergedRecord.put("age", 25); + mergedRecord.put("city", "New York"); + + when(mockRecordContext.constructEngineRecord(any(HoodieSchema.class), any(Object[].class))).thenReturn(mergedRecord); Review Comment: This needs to validate that the proper values are passed when constructing the value, otherwise we don't get good signal on whether the logic is correct ########## hudi-common/src/main/java/org/apache/hudi/common/table/read/PartialMergerWithKeepValues.java: ########## @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.read; + +import org.apache.hudi.common.engine.RecordContext; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; +import org.apache.hudi.common.util.VisibleForTesting; +import org.apache.hudi.common.util.collection.Pair; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Class to assist with merging two versions of the record that may contain partial updates using + * {@link org.apache.hudi.common.table.PartialUpdateMode#KEEP_VALUES} mode. + */ +public class PartialMergerWithKeepValues<T> implements Serializable { + private final Map<HoodieSchema, Map<String, Integer>> Review Comment: This mapping's keyset is referenced but the values are not. We can simplify this. ########## hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/TestBaseSparkInternalRowReaderContext.java: ########## @@ -141,6 +141,121 @@ void testConstructEngineRecordWithNullValueFromBase() { assertTrue(result.getBoolean(2)); } + @Test + void testConstructEngineRecordWithListOfValues() { + Object[] values = new Object[]{1, UTF8String.fromString("Alice"), true}; + InternalRow result = readerContext.getRecordContext().constructEngineRecord(SCHEMA, values); + + assertEquals(1, result.getInt(0)); + assertEquals("Alice", result.getString(1)); + assertTrue(result.getBoolean(2)); + } + + @Test + void testConstructEngineRecordWithNullValues() { + Object[] values = new Object[]{null, UTF8String.fromString("Bob"), null}; + InternalRow result = readerContext.getRecordContext().constructEngineRecord(SCHEMA, values); + + assertTrue(result.isNullAt(0)); + assertEquals("Bob", result.getString(1)); + assertTrue(result.isNullAt(2)); + } + + @Test + void testConstructEngineRecordWithMixedTypes() { + Object[] values = new Object[]{42, UTF8String.fromString("Carol"), false}; + InternalRow result = readerContext.getRecordContext().constructEngineRecord(SCHEMA, values); + + assertEquals(42, result.getInt(0)); + assertEquals("Carol", result.getString(1)); + assertFalse(result.getBoolean(2)); + } + + @Test + void testConstructEngineRecordWithEmptyValues() { + Object[] values = new Object[]{0, UTF8String.fromString(""), false}; + InternalRow result = readerContext.getRecordContext().constructEngineRecord(SCHEMA, values); + + assertEquals(0, result.getInt(0)); + assertEquals("", result.getString(1)); + assertFalse(result.getBoolean(2)); + } + + @Test + void testConstructEngineRecordWithValueCountMismatch() { + Object[] values = new Object[]{1, UTF8String.fromString("Alice")}; // Missing boolean value + + try { + readerContext.getRecordContext().constructEngineRecord(SCHEMA, values); + // Should not reach here + assertTrue(false, "Expected IllegalArgumentException"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Value count (2) does not match field count (3)")); Review Comment: Use assertThrows for this -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
