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 @@ -18,20 +18,13 @@

package org.apache.hudi.cli.commands;

import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.config.HoodieCommonConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
import org.apache.hudi.common.model.HoodieAvroRecordMerger;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
Expand All @@ -45,17 +38,21 @@
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.util.collection.ClosableIterator;
import org.apache.hudi.common.util.HoodieRecordUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ClosableIterator;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieMemoryConfig;

import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.springframework.shell.standard.ShellComponent;
import org.springframework.shell.standard.ShellMethod;
import org.springframework.shell.standard.ShellOption;
import scala.Tuple2;
import scala.Tuple3;

import java.io.IOException;
import java.util.ArrayList;
Expand All @@ -68,6 +65,9 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;

import scala.Tuple2;
import scala.Tuple3;

import static org.apache.hudi.common.util.ValidationUtils.checkArgument;

/**
Expand Down Expand Up @@ -221,7 +221,6 @@ public String showLogFileRecords(
.withSpillableMapBasePath(HoodieMemoryConfig.getDefaultSpillableMapBasePath())
.withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
.withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())
.withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
.withOptimizedLogBlocksScan(Boolean.parseBoolean(HoodieCompactionConfig.ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN.defaultValue()))
.build();
for (HoodieRecord hoodieRecord : scanner) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,8 @@
import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil;
import org.apache.hudi.common.config.HoodieCommonConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
import org.apache.hudi.common.model.HoodieAvroRecordMerger;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
Expand All @@ -40,7 +39,6 @@
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.common.util.HoodieRecordUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieMemoryConfig;
Expand Down Expand Up @@ -232,7 +230,6 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc
.withSpillableMapBasePath(HoodieMemoryConfig.getDefaultSpillableMapBasePath())
.withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
.withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())
.withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
.withOptimizedLogBlocksScan(Boolean.parseBoolean(HoodieCompactionConfig.ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN.defaultValue()))
.build();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordMerger;
import org.apache.hudi.common.model.MetadataValues;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
Expand Down Expand Up @@ -230,7 +231,12 @@ private static <R> HoodieData<HoodieRecord<R>> getExistingRecords(
/**
* Merge the incoming record with the matching existing record loaded via {@link HoodieMergedReadHandle}. The existing record is the latest version in the table.
*/
private static <R> Option<HoodieRecord<R>> mergeIncomingWithExistingRecord(HoodieRecord<R> incoming, HoodieRecord<R> existing, Schema writeSchema, HoodieWriteConfig config) throws IOException {
private static <R> Option<HoodieRecord<R>> mergeIncomingWithExistingRecord(
HoodieRecord<R> incoming,
HoodieRecord<R> existing,
Schema writeSchema,
HoodieWriteConfig config,
HoodieRecordMerger recordMerger) throws IOException {
Schema existingSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField());
Schema writeSchemaWithMetaFields = HoodieAvroUtils.addMetadataFields(writeSchema, config.allowOperationMetadataField());
// prepend the hoodie meta fields as the incoming record does not have them
Expand All @@ -239,7 +245,7 @@ private static <R> Option<HoodieRecord<R>> mergeIncomingWithExistingRecord(Hoodi
// after prepend the meta fields, convert the record back to the original payload
HoodieRecord incomingWithMetaFields = incomingPrepended
.wrapIntoHoodieRecordPayloadWithParams(writeSchema, config.getProps(), Option.empty(), config.allowOperationMetadataField(), Option.empty(), false, Option.empty());
Option<Pair<HoodieRecord, Schema>> mergeResult = config.getRecordMerger()
Option<Pair<HoodieRecord, Schema>> mergeResult = recordMerger
.merge(existing, existingSchema, incomingWithMetaFields, writeSchemaWithMetaFields, config.getProps());
if (mergeResult.isPresent()) {
// the merged record needs to be converted back to the original payload
Expand Down Expand Up @@ -270,6 +276,7 @@ public static <R> HoodieData<HoodieRecord<R>> mergeForPartitionUpdates(
// merged existing records with current locations being set
HoodieData<HoodieRecord<R>> existingRecords = getExistingRecords(partitionLocations, config, hoodieTable);

final HoodieRecordMerger recordMerger = config.getRecordMerger();
HoodieData<HoodieRecord<R>> taggedUpdatingRecords = updatingRecords.mapToPair(r -> Pair.of(r.getRecordKey(), r))
.leftOuterJoin(existingRecords.mapToPair(r -> Pair.of(r.getRecordKey(), r)))
.values().flatMap(entry -> {
Expand All @@ -286,7 +293,7 @@ public static <R> HoodieData<HoodieRecord<R>> mergeForPartitionUpdates(
return Collections.singletonList(getTaggedRecord(incoming.newInstance(existing.getKey()), Option.of(existing.getCurrentLocation()))).iterator();
}

Option<HoodieRecord<R>> mergedOpt = mergeIncomingWithExistingRecord(incoming, existing, writeSchema, config);
Option<HoodieRecord<R>> mergedOpt = mergeIncomingWithExistingRecord(incoming, existing, writeSchema, config, recordMerger);
if (!mergedOpt.isPresent()) {
// merge resulted in delete: force tag the incoming to the old partition
return Collections.singletonList(getTaggedRecord(incoming.newInstance(existing.getKey()), Option.of(existing.getCurrentLocation()))).iterator();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -456,7 +456,7 @@ public void performMergeDataValidationCheck(WriteStatus writeStatus) {
}

long oldNumWrites = 0;
try (HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(this.config.getRecordMerger().getRecordType()).getFileReader(hoodieTable.getHadoopConf(), oldFilePath)) {
try (HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(this.recordMerger.getRecordType()).getFileReader(hoodieTable.getHadoopConf(), oldFilePath)) {
oldNumWrites = reader.getTotalRecords();
} catch (IOException e) {
throw new HoodieUpsertException("Failed to check for merge data validation", e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.function.SerializableFunctionUnchecked;
import org.apache.hudi.common.model.HoodieAvroRecordMerger;
import org.apache.hudi.common.model.HoodieRecordMerger;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.util.HoodieRecordUtils;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
Expand All @@ -32,7 +32,6 @@

import java.time.Duration;
import java.time.Instant;
import java.util.Properties;

public abstract class BaseWriteHelper<T, I, K, O, R> extends ParallelismHelper<I> {

Expand Down Expand Up @@ -92,15 +91,9 @@ public I combineOnCondition(
* @return Collection of HoodieRecord already be deduplicated
*/
public I deduplicateRecords(I records, HoodieTable<T, I, K, O> table, int parallelism) {
HoodieRecordMerger recordMerger = table.getConfig().getRecordMerger();
HoodieRecordMerger recordMerger = HoodieRecordUtils.mergerToPreCombineMode(table.getConfig().getRecordMerger());
return deduplicateRecords(records, table.getIndex(), parallelism, table.getConfig().getSchema(), table.getConfig().getProps(), recordMerger);
}

public I deduplicateRecords(I records, HoodieIndex<?, ?> index, int parallelism, String schema, Properties props, HoodieRecordMerger merger) {
TypedProperties updatedProps = HoodieAvroRecordMerger.Config.withLegacyOperatingModePreCombining(props);
return doDeduplicateRecords(records, index, parallelism, schema, updatedProps, merger);
}

protected abstract I doDeduplicateRecords(
I records, HoodieIndex<?, ?> index, int parallelism, String schema, TypedProperties props, HoodieRecordMerger merger);
public abstract I deduplicateRecords(I records, HoodieIndex<?, ?> index, int parallelism, String schema, TypedProperties props, HoodieRecordMerger merger);
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ protected HoodieData<HoodieRecord<T>> tag(HoodieData<HoodieRecord<T>> dedupedRec
}

@Override
protected HoodieData<HoodieRecord<T>> doDeduplicateRecords(
public HoodieData<HoodieRecord<T>> deduplicateRecords(
HoodieData<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, String schemaStr, TypedProperties props, HoodieRecordMerger merger) {
boolean isIndexingGlobal = index.isGlobal();
final SerializableSchema schema = new SerializableSchema(schemaStr);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ protected List<HoodieRecord<T>> tag(List<HoodieRecord<T>> dedupedRecords, Hoodie
}

@Override
protected List<HoodieRecord<T>> doDeduplicateRecords(
public List<HoodieRecord<T>> deduplicateRecords(
List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, String schemaStr, TypedProperties props, HoodieRecordMerger merger) {
// If index used is global, then records are expected to differ in their partitionPath
Map<Object, List<HoodieRecord<T>>> keyedRecords = records.stream()
Expand All @@ -110,7 +110,7 @@ protected List<HoodieRecord<T>> doDeduplicateRecords(
// we cannot allow the user to change the key or partitionPath, since that will affect
// everything
// so pick it from one of the records.
boolean choosePrev = rec1 == reducedRecord;
boolean choosePrev = rec1.getData() == reducedRecord.getData();
HoodieKey reducedKey = choosePrev ? rec1.getKey() : rec2.getKey();
HoodieOperation operation = choosePrev ? rec1.getOperation() : rec2.getOperation();
HoodieRecord<T> hoodieRecord = reducedRecord.newInstance(reducedKey, operation);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ protected List<HoodieRecord<T>> tag(List<HoodieRecord<T>> dedupedRecords, Hoodie
}

@Override
protected List<HoodieRecord<T>> doDeduplicateRecords(
public List<HoodieRecord<T>> deduplicateRecords(
List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, String schemaStr, TypedProperties props, HoodieRecordMerger merger) {
boolean isIndexingGlobal = index.isGlobal();
Map<Object, List<Pair<Object, HoodieRecord<T>>>> keyedRecords = records.stream().map(record -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordMerger;
import org.apache.hudi.common.model.HoodieSparkRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ClosableIterator;
Expand Down Expand Up @@ -78,16 +77,16 @@ protected void executeBootstrap(HoodieBootstrapHandle<?, ?, ?, ?> bootstrapHandl
KeyGeneratorInterface keyGenerator,
String partitionPath,
Schema schema) throws Exception {
HoodieRecordMerger recordMerger = table.getConfig().getRecordMerger();
HoodieRecord.HoodieRecordType recordType = table.getConfig().getRecordMerger().getRecordType();

HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(recordMerger.getRecordType())
HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(recordType)
.getFileReader(table.getHadoopConf(), sourceFilePath);

HoodieExecutor<Void> executor = null;
try {
Function<HoodieRecord, HoodieRecord> transformer = record -> {
String recordKey = record.getRecordKey(schema, Option.of(keyGenerator));
return createNewMetadataBootstrapRecord(recordKey, partitionPath, recordMerger.getRecordType())
return createNewMetadataBootstrapRecord(recordKey, partitionPath, recordType)
// NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific
// payload pointing into a shared, mutable (underlying) buffer we get a clean copy of
// it since these records will be inserted into the queue later.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,15 +46,14 @@
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieAvroRecordMerger;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieOperation;
import org.apache.hudi.common.model.HoodiePreCombineAvroRecordMerger;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordMerger;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
Expand All @@ -80,7 +79,6 @@
import org.apache.hudi.common.util.ClusteringUtils;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.HoodieRecordUtils;
import org.apache.hudi.common.util.MarkerUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
Expand Down Expand Up @@ -487,11 +485,10 @@ private void testDeduplication(
// Global dedup should be done based on recordKey only
HoodieIndex index = mock(HoodieIndex.class);
when(index.isGlobal()).thenReturn(true);
HoodieRecordMerger recordMerger = HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName());
int dedupParallelism = records.getNumPartitions() + 100;
HoodieData<HoodieRecord<RawTripTestPayload>> dedupedRecsRdd =
(HoodieData<HoodieRecord<RawTripTestPayload>>) HoodieWriteHelper.newInstance()
.deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema(), writeConfig.getProps(), recordMerger);
.deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema(), writeConfig.getProps(), HoodiePreCombineAvroRecordMerger.INSTANCE);
List<HoodieRecord<RawTripTestPayload>> dedupedRecs = dedupedRecsRdd.collectAsList();
assertEquals(records.getNumPartitions(), dedupedRecsRdd.getNumPartitions());
assertEquals(1, dedupedRecs.size());
Expand All @@ -503,7 +500,7 @@ private void testDeduplication(
when(index.isGlobal()).thenReturn(false);
dedupedRecsRdd =
(HoodieData<HoodieRecord<RawTripTestPayload>>) HoodieWriteHelper.newInstance()
.deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema(), writeConfig.getProps(), recordMerger);
.deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema(), writeConfig.getProps(), HoodiePreCombineAvroRecordMerger.INSTANCE);
dedupedRecs = dedupedRecsRdd.collectAsList();
assertEquals(records.getNumPartitions(), dedupedRecsRdd.getNumPartitions());
assertEquals(2, dedupedRecs.size());
Expand Down Expand Up @@ -555,11 +552,10 @@ private void testDeduplicationKeepOperation(
// Global dedup should be done based on recordKey only
HoodieIndex index = mock(HoodieIndex.class);
when(index.isGlobal()).thenReturn(true);
HoodieRecordMerger recordMerger = HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName());
int dedupParallelism = records.getNumPartitions() + 100;
HoodieData<HoodieRecord<RawTripTestPayload>> dedupedRecsRdd =
(HoodieData<HoodieRecord<RawTripTestPayload>>) HoodieWriteHelper.newInstance()
.deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema(), writeConfig.getProps(), recordMerger);
.deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema(), writeConfig.getProps(), HoodiePreCombineAvroRecordMerger.INSTANCE);
List<HoodieRecord<RawTripTestPayload>> dedupedRecs = dedupedRecsRdd.collectAsList();
assertEquals(dedupedRecs.get(0).getOperation(), recordThree.getOperation());

Expand Down
Loading