Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -233,8 +233,11 @@ public HoodieMetadataPayload(Option<GenericRecord> recordOpt) {
columnStatMetadata = HoodieMetadataColumnStats.newBuilder(METADATA_COLUMN_STATS_BUILDER_STUB.get())
.setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_FILE_NAME))
.setColumnName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_COLUMN_NAME))
.setMinValue(columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE))
.setMaxValue(columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE))
// AVRO-2377 1.9.2 Modified the type of org.apache.avro.Schema#FIELD_RESERVED to Collections.unmodifiableSet.
// This causes Kryo to fail when deserializing a GenericRecord, See HUDI-5484.
// We should avoid using GenericRecord and convert GenericRecord into a serializable type.
.setMinValue(wrapStatisticValue(unwrapStatisticValueWrapper(columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE))))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's add a comment explaining why we need to do that here

.setMaxValue(wrapStatisticValue(unwrapStatisticValueWrapper(columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE))))
.setValueCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_VALUE_COUNT))
.setNullCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_NULL_COUNT))
.setTotalSize((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_TOTAL_SIZE))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,15 +22,18 @@
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.common.config.HoodieStorageConfig;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieColumnRangeMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SerializationUtils;
import org.apache.hudi.common.util.collection.ImmutablePair;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner;
import org.apache.hudi.metadata.HoodieMetadataPayload;
import org.apache.hudi.table.BulkInsertPartitioner;

import org.apache.avro.Conversions;
Expand All @@ -39,6 +42,7 @@
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
Expand All @@ -59,6 +63,7 @@
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;

import java.io.IOException;
import java.math.BigDecimal;
import java.time.LocalDate;
import java.util.ArrayList;
Expand Down Expand Up @@ -323,4 +328,28 @@ private static Stream<Arguments> testAutoModifyParquetWriteLegacyFormatParameter
{false, true, true}, {false, false, false}
}).map(Arguments::of);
}

@Test
public void testSerHoodieMetadataPayload() throws IOException {
String partitionPath = "2022/10/01";
String fileName = "file.parquet";
String targetColName = "c1";

HoodieColumnRangeMetadata<Comparable> columnStatsRecord =
HoodieColumnRangeMetadata.<Comparable>create(fileName, targetColName, 0, 500, 0, 100, 12345, 12345);

HoodieRecord<HoodieMetadataPayload> hoodieMetadataPayload =
HoodieMetadataPayload.createColumnStatsRecords(partitionPath, Collections.singletonList(columnStatsRecord), false)
.findFirst().get();

IndexedRecord record = hoodieMetadataPayload.getData().getInsertValue(null).get();
byte[] recordToBytes = HoodieAvroUtils.indexedRecordToBytes(record);
GenericRecord genericRecord = HoodieAvroUtils.bytesToAvro(recordToBytes, record.getSchema());

HoodieMetadataPayload genericRecordHoodieMetadataPayload = new HoodieMetadataPayload(Option.of(genericRecord));
byte[] bytes = SerializationUtils.serialize(genericRecordHoodieMetadataPayload);
HoodieMetadataPayload deserGenericRecordHoodieMetadataPayload = SerializationUtils.deserialize(bytes);

assertEquals(genericRecordHoodieMetadataPayload, deserGenericRecordHoodieMetadataPayload);
}
}