[HUDI-1771] Propagate CDC format for hoodie (#3285)
This commit is contained in:
@@ -76,6 +76,17 @@ public class FlinkOptions extends HoodieConfig {
|
||||
.withDescription("The default partition name in case the dynamic partition"
|
||||
+ " column value is null/empty string");
|
||||
|
||||
public static final ConfigOption<Boolean> CHANGELOG_ENABLED = ConfigOptions
|
||||
.key("changelog.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Whether to keep all the intermediate changes, "
|
||||
+ "we try to keep all the changes of a record when enabled:\n"
|
||||
+ "1). The sink accept the UPDATE_BEFORE message;\n"
|
||||
+ "2). The source try to emit every changes of a record.\n"
|
||||
+ "The semantics is best effort because the compaction job would finally merge all changes of a record into one.\n"
|
||||
+ " default false to have UPSERT semantics");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Metadata table Options
|
||||
// ------------------------------------------------------------------------
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.sink;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
@@ -360,23 +361,26 @@ public class StreamWriteFunction<K, I, O>
|
||||
private final String key; // record key
|
||||
private final String instant; // 'U' or 'I'
|
||||
private final HoodieRecordPayload<?> data; // record payload
|
||||
private final HoodieOperation operation; // operation
|
||||
|
||||
private DataItem(String key, String instant, HoodieRecordPayload<?> data) {
|
||||
private DataItem(String key, String instant, HoodieRecordPayload<?> data, HoodieOperation operation) {
|
||||
this.key = key;
|
||||
this.instant = instant;
|
||||
this.data = data;
|
||||
this.operation = operation;
|
||||
}
|
||||
|
||||
public static DataItem fromHoodieRecord(HoodieRecord<?> record) {
|
||||
return new DataItem(
|
||||
record.getRecordKey(),
|
||||
record.getCurrentLocation().getInstantTime(),
|
||||
record.getData());
|
||||
record.getData(),
|
||||
record.getOperation());
|
||||
}
|
||||
|
||||
public HoodieRecord<?> toHoodieRecord(String partitionPath) {
|
||||
HoodieKey hoodieKey = new HoodieKey(this.key, partitionPath);
|
||||
HoodieRecord<?> record = new HoodieRecord<>(hoodieKey, data);
|
||||
HoodieRecord<?> record = new HoodieRecord<>(hoodieKey, data, operation);
|
||||
HoodieRecordLocation loc = new HoodieRecordLocation(instant, null);
|
||||
record.setCurrentLocation(loc);
|
||||
return record;
|
||||
@@ -417,7 +421,7 @@ public class StreamWriteFunction<K, I, O>
|
||||
public void preWrite(List<HoodieRecord> records) {
|
||||
// rewrite the first record with expected fileID
|
||||
HoodieRecord<?> first = records.get(0);
|
||||
HoodieRecord<?> record = new HoodieRecord<>(first.getKey(), first.getData());
|
||||
HoodieRecord<?> record = new HoodieRecord<>(first.getKey(), first.getData(), first.getOperation());
|
||||
HoodieRecordLocation newLoc = new HoodieRecordLocation(first.getCurrentLocation().getInstantTime(), fileID);
|
||||
record.setCurrentLocation(newLoc);
|
||||
|
||||
|
||||
@@ -203,7 +203,7 @@ public class BootstrapFunction<I, O extends HoodieRecord>
|
||||
.filter(logFile -> isValidFile(logFile.getFileStatus()))
|
||||
.map(logFile -> logFile.getPath().toString())
|
||||
.collect(toList());
|
||||
HoodieMergedLogRecordScanner scanner = FormatUtils.scanLog(logPaths, schema, latestCommitTime.get().getTimestamp(),
|
||||
HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(logPaths, schema, latestCommitTime.get().getTimestamp(),
|
||||
writeConfig, hadoopConf);
|
||||
|
||||
try {
|
||||
|
||||
@@ -71,7 +71,7 @@ public class BulkInsertWriterHelper {
|
||||
this.taskPartitionId = taskPartitionId;
|
||||
this.taskId = taskId;
|
||||
this.taskEpochId = taskEpochId;
|
||||
this.rowType = addMetadataFields(rowType); // patch up with metadata fields
|
||||
this.rowType = addMetadataFields(rowType, writeConfig.allowOperationMetadataField()); // patch up with metadata fields
|
||||
this.arePartitionRecordsSorted = conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SORT_BY_PARTITION);
|
||||
this.fileIdPrefix = UUID.randomUUID().toString();
|
||||
this.keyGen = RowDataKeyGen.instance(conf, rowType);
|
||||
@@ -141,7 +141,7 @@ public class BulkInsertWriterHelper {
|
||||
/**
|
||||
* Adds the Hoodie metadata fields to the given row type.
|
||||
*/
|
||||
private static RowType addMetadataFields(RowType rowType) {
|
||||
private static RowType addMetadataFields(RowType rowType, boolean withOperationField) {
|
||||
List<RowType.RowField> mergedFields = new ArrayList<>();
|
||||
|
||||
LogicalType metadataFieldType = DataTypes.STRING().getLogicalType();
|
||||
@@ -161,6 +161,13 @@ public class BulkInsertWriterHelper {
|
||||
mergedFields.add(recordKeyField);
|
||||
mergedFields.add(partitionPathField);
|
||||
mergedFields.add(fileNameField);
|
||||
|
||||
if (withOperationField) {
|
||||
RowType.RowField operationField =
|
||||
new RowType.RowField(HoodieRecord.OPERATION_METADATA_FIELD, metadataFieldType, "operation");
|
||||
mergedFields.add(operationField);
|
||||
}
|
||||
|
||||
mergedFields.addAll(rowType.getFields());
|
||||
|
||||
return new RowType(false, mergedFields);
|
||||
|
||||
@@ -20,8 +20,6 @@ package org.apache.hudi.sink.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
|
||||
import org.apache.flink.api.common.functions.AbstractRichFunction;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
@@ -57,26 +55,14 @@ public class CompactionPlanSourceFunction extends AbstractRichFunction implement
|
||||
/**
|
||||
* Compaction instant time.
|
||||
*/
|
||||
private String compactionInstantTime;
|
||||
|
||||
/**
|
||||
* Hoodie flink table.
|
||||
*/
|
||||
private HoodieFlinkTable<?> table;
|
||||
private final String compactionInstantTime;
|
||||
|
||||
/**
|
||||
* The compaction plan.
|
||||
*/
|
||||
private HoodieCompactionPlan compactionPlan;
|
||||
private final HoodieCompactionPlan compactionPlan;
|
||||
|
||||
/**
|
||||
* Hoodie instant.
|
||||
*/
|
||||
private HoodieInstant instant;
|
||||
|
||||
public CompactionPlanSourceFunction(HoodieFlinkTable<?> table, HoodieInstant instant, HoodieCompactionPlan compactionPlan, String compactionInstantTime) {
|
||||
this.table = table;
|
||||
this.instant = instant;
|
||||
public CompactionPlanSourceFunction(HoodieCompactionPlan compactionPlan, String compactionInstantTime) {
|
||||
this.compactionPlan = compactionPlan;
|
||||
this.compactionInstantTime = compactionInstantTime;
|
||||
}
|
||||
|
||||
@@ -133,9 +133,8 @@ public class HoodieFlinkCompactor {
|
||||
|
||||
// Mark instant as compaction inflight
|
||||
table.getActiveTimeline().transitionCompactionRequestedToInflight(instant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
env.addSource(new CompactionPlanSourceFunction(table, instant, compactionPlan, compactionInstantTime))
|
||||
env.addSource(new CompactionPlanSourceFunction(compactionPlan, compactionInstantTime))
|
||||
.name("compaction_source")
|
||||
.uid("uid_compaction_source")
|
||||
.rebalance()
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.sink.transform;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
@@ -34,7 +35,6 @@ import org.apache.flink.api.common.functions.RichMapFunction;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.types.RowKind;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@@ -108,9 +108,9 @@ public class RowDataToHoodieFunction<I extends RowData, O extends HoodieRecord>
|
||||
private HoodieRecord toHoodieRecord(I record) throws Exception {
|
||||
GenericRecord gr = (GenericRecord) this.converter.convert(this.avroSchema, record);
|
||||
final HoodieKey hoodieKey = keyGenerator.getKey(gr);
|
||||
// nullify the payload insert data to mark the record as a DELETE
|
||||
final boolean isDelete = record.getRowKind() == RowKind.DELETE;
|
||||
HoodieRecordPayload payload = payloadCreation.createPayload(gr, isDelete);
|
||||
return new HoodieRecord<>(hoodieKey, payload);
|
||||
|
||||
HoodieRecordPayload payload = payloadCreation.createPayload(gr);
|
||||
HoodieOperation operation = HoodieOperation.fromValue(record.getRowKind().toByteValue());
|
||||
return new HoodieRecord<>(hoodieKey, payload, operation);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -86,6 +86,7 @@ public class HiveSyncContext {
|
||||
hiveSyncConfig.decodePartition = conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING);
|
||||
hiveSyncConfig.skipROSuffix = conf.getBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX);
|
||||
hiveSyncConfig.assumeDatePartitioning = conf.getBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION);
|
||||
hiveSyncConfig.withOperationField = conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED);
|
||||
return hiveSyncConfig;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -71,13 +71,12 @@ public class PayloadCreation implements Serializable {
|
||||
return new PayloadCreation(shouldCombine, constructor, preCombineField);
|
||||
}
|
||||
|
||||
public HoodieRecordPayload<?> createPayload(GenericRecord record, boolean isDelete) throws Exception {
|
||||
public HoodieRecordPayload<?> createPayload(GenericRecord record) throws Exception {
|
||||
if (shouldCombine) {
|
||||
ValidationUtils.checkState(preCombineField != null);
|
||||
Comparable<?> orderingVal = (Comparable<?>) HoodieAvroUtils.getNestedFieldVal(record,
|
||||
preCombineField, false);
|
||||
return (HoodieRecordPayload<?>) constructor.newInstance(
|
||||
isDelete ? null : record, orderingVal);
|
||||
return (HoodieRecordPayload<?>) constructor.newInstance(record, orderingVal);
|
||||
} else {
|
||||
return (HoodieRecordPayload<?>) this.constructor.newInstance(Option.of(record));
|
||||
}
|
||||
|
||||
@@ -34,6 +34,7 @@ import org.apache.hudi.sink.compact.CompactionPlanEvent;
|
||||
import org.apache.hudi.sink.compact.CompactionPlanOperator;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignOperator;
|
||||
import org.apache.hudi.util.ChangelogModes;
|
||||
import org.apache.hudi.sink.transform.RowDataToHoodieFunctions;
|
||||
import org.apache.hudi.table.format.FilePathUtils;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
@@ -52,7 +53,6 @@ import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode$;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.types.RowKind;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
@@ -185,12 +185,11 @@ public class HoodieTableSink implements DynamicTableSink, SupportsPartitioning,
|
||||
|
||||
@Override
|
||||
public ChangelogMode getChangelogMode(ChangelogMode changelogMode) {
|
||||
// ignore RowKind.UPDATE_BEFORE
|
||||
return ChangelogMode.newBuilder()
|
||||
.addContainedKind(RowKind.DELETE)
|
||||
.addContainedKind(RowKind.INSERT)
|
||||
.addContainedKind(RowKind.UPDATE_AFTER)
|
||||
.build();
|
||||
if (conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)) {
|
||||
return ChangelogModes.FULL;
|
||||
} else {
|
||||
return ChangelogModes.UPSERT;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -37,6 +37,7 @@ import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadTableState;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.ChangelogModes;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
@@ -196,7 +197,12 @@ public class HoodieTableSource implements
|
||||
|
||||
@Override
|
||||
public ChangelogMode getChangelogMode() {
|
||||
return ChangelogMode.insertOnly();
|
||||
return conf.getBoolean(FlinkOptions.READ_AS_STREAMING)
|
||||
&& !conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)
|
||||
? ChangelogModes.FULL
|
||||
// when all the changes are persisted or read as batch,
|
||||
// use INSERT mode.
|
||||
: ChangelogMode.insertOnly();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -309,7 +315,7 @@ public class HoodieTableSource implements
|
||||
return new CollectionInputFormat<>(Collections.emptyList(), null);
|
||||
}
|
||||
|
||||
TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient);
|
||||
TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED));
|
||||
final Schema tableAvroSchema;
|
||||
try {
|
||||
tableAvroSchema = schemaUtil.getTableAvroSchema();
|
||||
|
||||
@@ -19,19 +19,32 @@
|
||||
package org.apache.hudi.table.format;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.GenericRecordBuilder;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.types.RowKind;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
@@ -45,6 +58,39 @@ public class FormatUtils {
|
||||
private FormatUtils() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets up the row kind to the row data {@code rowData} from the resolved operation.
|
||||
*/
|
||||
public static void setRowKind(RowData rowData, IndexedRecord record, int index) {
|
||||
if (index == -1) {
|
||||
return;
|
||||
}
|
||||
rowData.setRowKind(getRowKind(record, index));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the RowKind of the given record, never null.
|
||||
* Returns RowKind.INSERT when the given field value not found.
|
||||
*/
|
||||
private static RowKind getRowKind(IndexedRecord record, int index) {
|
||||
Object val = record.get(index);
|
||||
if (val == null) {
|
||||
return RowKind.INSERT;
|
||||
}
|
||||
final HoodieOperation operation = HoodieOperation.fromName(val.toString());
|
||||
if (HoodieOperation.isInsert(operation)) {
|
||||
return RowKind.INSERT;
|
||||
} else if (HoodieOperation.isUpdateBefore(operation)) {
|
||||
return RowKind.UPDATE_BEFORE;
|
||||
} else if (HoodieOperation.isUpdateAfter(operation)) {
|
||||
return RowKind.UPDATE_AFTER;
|
||||
} else if (HoodieOperation.isDelete(operation)) {
|
||||
return RowKind.DELETE;
|
||||
} else {
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
public static GenericRecord buildAvroRecordBySchema(
|
||||
IndexedRecord record,
|
||||
Schema requiredSchema,
|
||||
@@ -57,10 +103,11 @@ public class FormatUtils {
|
||||
return recordBuilder.build();
|
||||
}
|
||||
|
||||
public static HoodieMergedLogRecordScanner scanLog(
|
||||
public static HoodieMergedLogRecordScanner logScanner(
|
||||
MergeOnReadInputSplit split,
|
||||
Schema logSchema,
|
||||
Configuration config) {
|
||||
Configuration config,
|
||||
boolean withOperationField) {
|
||||
FileSystem fs = FSUtils.getFs(split.getTablePath(), config);
|
||||
return HoodieMergedLogRecordScanner.newBuilder()
|
||||
.withFileSystem(fs)
|
||||
@@ -81,10 +128,88 @@ public class FormatUtils {
|
||||
config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH))
|
||||
.withInstantRange(split.getInstantRange())
|
||||
.withOperationField(withOperationField)
|
||||
.build();
|
||||
}
|
||||
|
||||
public static HoodieMergedLogRecordScanner scanLog(
|
||||
private static HoodieUnMergedLogRecordScanner unMergedLogScanner(
|
||||
MergeOnReadInputSplit split,
|
||||
Schema logSchema,
|
||||
Configuration config,
|
||||
HoodieUnMergedLogRecordScanner.LogRecordScannerCallback callback) {
|
||||
FileSystem fs = FSUtils.getFs(split.getTablePath(), config);
|
||||
return HoodieUnMergedLogRecordScanner.newBuilder()
|
||||
.withFileSystem(fs)
|
||||
.withBasePath(split.getTablePath())
|
||||
.withLogFilePaths(split.getLogPaths().get())
|
||||
.withReaderSchema(logSchema)
|
||||
.withLatestInstantTime(split.getLatestCommit())
|
||||
.withReadBlocksLazily(
|
||||
string2Boolean(
|
||||
config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)))
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(
|
||||
config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
|
||||
.withInstantRange(split.getInstantRange())
|
||||
.withLogRecordScannerCallback(callback)
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to read and buffer the records in the unMerged log record scanner.
|
||||
*/
|
||||
public static class BoundedMemoryRecords {
|
||||
// Log Record unmerged scanner
|
||||
private final HoodieUnMergedLogRecordScanner scanner;
|
||||
|
||||
// Executor that runs the above producers in parallel
|
||||
private final BoundedInMemoryExecutor<HoodieRecord<?>, HoodieRecord<?>, ?> executor;
|
||||
|
||||
// Iterator for the buffer consumer
|
||||
private final Iterator<HoodieRecord<?>> iterator;
|
||||
|
||||
public BoundedMemoryRecords(
|
||||
MergeOnReadInputSplit split,
|
||||
Schema logSchema,
|
||||
Configuration hadoopConf) {
|
||||
this.executor = new BoundedInMemoryExecutor<>(
|
||||
HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes(new JobConf(hadoopConf)),
|
||||
getParallelProducers(),
|
||||
Option.empty(),
|
||||
x -> x,
|
||||
new DefaultSizeEstimator<>());
|
||||
// Consumer of this record reader
|
||||
this.iterator = this.executor.getQueue().iterator();
|
||||
this.scanner = FormatUtils.unMergedLogScanner(split, logSchema, hadoopConf,
|
||||
record -> executor.getQueue().insertRecord(record));
|
||||
// Start reading and buffering
|
||||
this.executor.startProducers();
|
||||
}
|
||||
|
||||
public Iterator<HoodieRecord<?>> getRecordsIterator() {
|
||||
return this.iterator;
|
||||
}
|
||||
|
||||
/**
|
||||
* Setup log and parquet reading in parallel. Both write to central buffer.
|
||||
*/
|
||||
private List<BoundedInMemoryQueueProducer<HoodieRecord<?>>> getParallelProducers() {
|
||||
List<BoundedInMemoryQueueProducer<HoodieRecord<?>>> producers = new ArrayList<>();
|
||||
producers.add(new FunctionBasedQueueProducer<>(buffer -> {
|
||||
scanner.scan();
|
||||
return null;
|
||||
}));
|
||||
return producers;
|
||||
}
|
||||
|
||||
public void close() {
|
||||
this.executor.shutdownNow();
|
||||
}
|
||||
}
|
||||
|
||||
public static HoodieMergedLogRecordScanner logScanner(
|
||||
List<String> logPaths,
|
||||
Schema logSchema,
|
||||
String latestInstantTime,
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.table.format.mor;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieOperation;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.log.InstantRange;
|
||||
@@ -176,7 +177,11 @@ public class MergeOnReadInputFormat
|
||||
}
|
||||
} else if (!split.getBasePath().isPresent()) {
|
||||
// log files only
|
||||
this.iterator = new LogFileOnlyIterator(getLogFileIterator(split));
|
||||
if (conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED)) {
|
||||
this.iterator = new LogFileOnlyIterator(getUnMergedLogFileIterator(split));
|
||||
} else {
|
||||
this.iterator = new LogFileOnlyIterator(getLogFileIterator(split));
|
||||
}
|
||||
} else if (split.getMergeType().equals(FlinkOptions.REALTIME_SKIP_MERGE)) {
|
||||
this.iterator = new SkipMergeIterator(
|
||||
getRequiredSchemaReader(split.getBasePath().get()),
|
||||
@@ -190,6 +195,9 @@ public class MergeOnReadInputFormat
|
||||
new Schema.Parser().parse(this.tableState.getAvroSchema()),
|
||||
new Schema.Parser().parse(this.tableState.getRequiredAvroSchema()),
|
||||
this.requiredPos,
|
||||
this.emitDelete,
|
||||
this.conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED),
|
||||
this.tableState.getOperationPos(),
|
||||
getFullSchemaReader(split.getBasePath().get()));
|
||||
} else {
|
||||
throw new HoodieException("Unable to select an Iterator to read the Hoodie MOR File Split for "
|
||||
@@ -298,7 +306,7 @@ public class MergeOnReadInputFormat
|
||||
final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema);
|
||||
final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter =
|
||||
AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType());
|
||||
final HoodieMergedLogRecordScanner scanner = FormatUtils.scanLog(split, tableSchema, hadoopConf);
|
||||
final HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(split, tableSchema, hadoopConf, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED));
|
||||
final Iterator<String> logRecordsKeyIterator = scanner.getRecords().keySet().iterator();
|
||||
final int[] pkOffset = tableState.getPkOffsetsInRequired();
|
||||
// flag saying whether the pk semantics has been dropped by user specified
|
||||
@@ -335,18 +343,20 @@ public class MergeOnReadInputFormat
|
||||
}
|
||||
delete.setRowKind(RowKind.DELETE);
|
||||
|
||||
this.currentRecord = delete;
|
||||
this.currentRecord = delete;
|
||||
return true;
|
||||
}
|
||||
// skipping if the condition is unsatisfied
|
||||
// continue;
|
||||
} else {
|
||||
final IndexedRecord avroRecord = curAvroRecord.get();
|
||||
GenericRecord requiredAvroRecord = buildAvroRecordBySchema(
|
||||
curAvroRecord.get(),
|
||||
avroRecord,
|
||||
requiredSchema,
|
||||
requiredPos,
|
||||
recordBuilder);
|
||||
currentRecord = (RowData) avroToRowDataConverter.convert(requiredAvroRecord);
|
||||
FormatUtils.setRowKind(currentRecord, avroRecord, tableState.getOperationPos());
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -365,6 +375,55 @@ public class MergeOnReadInputFormat
|
||||
};
|
||||
}
|
||||
|
||||
private ClosableIterator<RowData> getUnMergedLogFileIterator(MergeOnReadInputSplit split) {
|
||||
final Schema tableSchema = new Schema.Parser().parse(tableState.getAvroSchema());
|
||||
final Schema requiredSchema = new Schema.Parser().parse(tableState.getRequiredAvroSchema());
|
||||
final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema);
|
||||
final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter =
|
||||
AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType());
|
||||
final FormatUtils.BoundedMemoryRecords records = new FormatUtils.BoundedMemoryRecords(split, tableSchema, hadoopConf);
|
||||
final Iterator<HoodieRecord<?>> recordsIterator = records.getRecordsIterator();
|
||||
|
||||
return new ClosableIterator<RowData>() {
|
||||
private RowData currentRecord;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
while (recordsIterator.hasNext()) {
|
||||
Option<IndexedRecord> curAvroRecord = null;
|
||||
final HoodieRecord<?> hoodieRecord = recordsIterator.next();
|
||||
try {
|
||||
curAvroRecord = hoodieRecord.getData().getInsertValue(tableSchema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Get avro insert value error for key: " + hoodieRecord.getRecordKey(), e);
|
||||
}
|
||||
if (curAvroRecord.isPresent()) {
|
||||
final IndexedRecord avroRecord = curAvroRecord.get();
|
||||
GenericRecord requiredAvroRecord = buildAvroRecordBySchema(
|
||||
avroRecord,
|
||||
requiredSchema,
|
||||
requiredPos,
|
||||
recordBuilder);
|
||||
currentRecord = (RowData) avroToRowDataConverter.convert(requiredAvroRecord);
|
||||
FormatUtils.setRowKind(currentRecord, avroRecord, tableState.getOperationPos());
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData next() {
|
||||
return currentRecord;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
records.close();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
@@ -544,6 +603,8 @@ public class MergeOnReadInputFormat
|
||||
private final Schema tableSchema;
|
||||
private final Schema requiredSchema;
|
||||
private final int[] requiredPos;
|
||||
private final boolean emitDelete;
|
||||
private final int operationPos;
|
||||
private final RowDataToAvroConverters.RowDataToAvroConverter rowDataToAvroConverter;
|
||||
private final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter;
|
||||
private final GenericRecordBuilder recordBuilder;
|
||||
@@ -557,7 +618,7 @@ public class MergeOnReadInputFormat
|
||||
// refactor it out once FLINK-22370 is resolved.
|
||||
private boolean readLogs = false;
|
||||
|
||||
private Set<String> keyToSkip = new HashSet<>();
|
||||
private final Set<String> keyToSkip = new HashSet<>();
|
||||
|
||||
private RowData currentRecord;
|
||||
|
||||
@@ -569,13 +630,18 @@ public class MergeOnReadInputFormat
|
||||
Schema tableSchema,
|
||||
Schema requiredSchema,
|
||||
int[] requiredPos,
|
||||
boolean emitDelete,
|
||||
boolean withOperationField,
|
||||
int operationPos,
|
||||
ParquetColumnarRowSplitReader reader) { // the reader should be with full schema
|
||||
this.tableSchema = tableSchema;
|
||||
this.reader = reader;
|
||||
this.scanner = FormatUtils.scanLog(split, tableSchema, hadoopConf);
|
||||
this.scanner = FormatUtils.logScanner(split, tableSchema, hadoopConf, withOperationField);
|
||||
this.logKeysIterator = scanner.getRecords().keySet().iterator();
|
||||
this.requiredSchema = requiredSchema;
|
||||
this.requiredPos = requiredPos;
|
||||
this.emitDelete = emitDelete;
|
||||
this.operationPos = operationPos;
|
||||
this.recordBuilder = new GenericRecordBuilder(requiredSchema);
|
||||
this.rowDataToAvroConverter = RowDataToAvroConverters.createConverter(tableRowType);
|
||||
this.avroToRowDataConverter = AvroToRowDataConverters.createRowConverter(requiredRowType);
|
||||
@@ -602,12 +668,13 @@ public class MergeOnReadInputFormat
|
||||
// deleted
|
||||
continue;
|
||||
} else {
|
||||
GenericRecord record = buildAvroRecordBySchema(
|
||||
GenericRecord avroRecord = buildAvroRecordBySchema(
|
||||
mergedAvroRecord.get(),
|
||||
requiredSchema,
|
||||
requiredPos,
|
||||
recordBuilder);
|
||||
this.currentRecord = (RowData) avroToRowDataConverter.convert(record);
|
||||
this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord);
|
||||
FormatUtils.setRowKind(this.currentRecord, mergedAvroRecord.get(), this.operationPos);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@@ -620,16 +687,16 @@ public class MergeOnReadInputFormat
|
||||
while (logKeysIterator.hasNext()) {
|
||||
final String curKey = logKeysIterator.next();
|
||||
if (!keyToSkip.contains(curKey)) {
|
||||
Option<IndexedRecord> insertAvroRecord =
|
||||
scanner.getRecords().get(curKey).getData().getInsertValue(tableSchema);
|
||||
Option<IndexedRecord> insertAvroRecord = getInsertValue(curKey);
|
||||
if (insertAvroRecord.isPresent()) {
|
||||
// the record is a DELETE if insertAvroRecord not present, skipping
|
||||
GenericRecord requiredAvroRecord = buildAvroRecordBySchema(
|
||||
GenericRecord avroRecord = buildAvroRecordBySchema(
|
||||
insertAvroRecord.get(),
|
||||
requiredSchema,
|
||||
requiredPos,
|
||||
recordBuilder);
|
||||
this.currentRecord = (RowData) avroToRowDataConverter.convert(requiredAvroRecord);
|
||||
this.currentRecord = (RowData) avroToRowDataConverter.convert(avroRecord);
|
||||
FormatUtils.setRowKind(this.currentRecord, insertAvroRecord.get(), this.operationPos);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@@ -637,6 +704,14 @@ public class MergeOnReadInputFormat
|
||||
return true;
|
||||
}
|
||||
|
||||
private Option<IndexedRecord> getInsertValue(String curKey) throws IOException {
|
||||
final HoodieRecord<?> record = scanner.getRecords().get(curKey);
|
||||
if (!emitDelete && HoodieOperation.isDelete(record.getOperation())) {
|
||||
return Option.empty();
|
||||
}
|
||||
return record.getData().getInsertValue(tableSchema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowData nextRecord() {
|
||||
return currentRecord;
|
||||
@@ -655,8 +730,12 @@ public class MergeOnReadInputFormat
|
||||
private Option<IndexedRecord> mergeRowWithLog(
|
||||
RowData curRow,
|
||||
String curKey) throws IOException {
|
||||
final HoodieRecord<?> record = scanner.getRecords().get(curKey);
|
||||
if (!emitDelete && HoodieOperation.isDelete(record.getOperation())) {
|
||||
return Option.empty();
|
||||
}
|
||||
GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow);
|
||||
return scanner.getRecords().get(curKey).getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema);
|
||||
return record.getData().combineAndGetUpdateValue(historyAvroRecord, tableSchema);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -18,6 +18,8 @@
|
||||
|
||||
package org.apache.hudi.table.format.mor;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
@@ -38,6 +40,7 @@ public class MergeOnReadTableState implements Serializable {
|
||||
private final String requiredAvroSchema;
|
||||
private final List<MergeOnReadInputSplit> inputSplits;
|
||||
private final String[] pkFields;
|
||||
private final int operationPos;
|
||||
|
||||
public MergeOnReadTableState(
|
||||
RowType rowType,
|
||||
@@ -52,6 +55,7 @@ public class MergeOnReadTableState implements Serializable {
|
||||
this.requiredAvroSchema = requiredAvroSchema;
|
||||
this.inputSplits = inputSplits;
|
||||
this.pkFields = pkFields;
|
||||
this.operationPos = rowType.getFieldIndex(HoodieRecord.OPERATION_METADATA_FIELD);
|
||||
}
|
||||
|
||||
public RowType getRowType() {
|
||||
@@ -74,6 +78,10 @@ public class MergeOnReadTableState implements Serializable {
|
||||
return inputSplits;
|
||||
}
|
||||
|
||||
public int getOperationPos() {
|
||||
return operationPos;
|
||||
}
|
||||
|
||||
public int[] getRequiredPositions() {
|
||||
final List<String> fieldNames = rowType.getFieldNames();
|
||||
return requiredRowType.getFieldNames().stream()
|
||||
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import org.apache.flink.table.connector.ChangelogMode;
|
||||
import org.apache.flink.types.RowKind;
|
||||
|
||||
/**
|
||||
* Utilities for all kinds of common {@link org.apache.flink.table.connector.ChangelogMode}s.
|
||||
*/
|
||||
public class ChangelogModes {
|
||||
public static final ChangelogMode FULL = ChangelogMode.newBuilder()
|
||||
.addContainedKind(RowKind.INSERT)
|
||||
.addContainedKind(RowKind.UPDATE_BEFORE)
|
||||
.addContainedKind(RowKind.UPDATE_AFTER)
|
||||
.addContainedKind(RowKind.DELETE)
|
||||
.build();
|
||||
|
||||
/**
|
||||
* Change log mode that ignores UPDATE_BEFORE, e.g UPSERT.
|
||||
*/
|
||||
public static final ChangelogMode UPSERT = ChangelogMode.newBuilder()
|
||||
.addContainedKind(RowKind.INSERT)
|
||||
.addContainedKind(RowKind.UPDATE_AFTER)
|
||||
.addContainedKind(RowKind.DELETE)
|
||||
.build();
|
||||
|
||||
private ChangelogModes() {
|
||||
}
|
||||
}
|
||||
@@ -186,6 +186,7 @@ public class StreamerUtil {
|
||||
.build())
|
||||
.withEmbeddedTimelineServerReuseEnabled(true) // make write client embedded timeline service singleton
|
||||
.withAutoCommit(false)
|
||||
.withAllowOperationMetadataField(conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED))
|
||||
.withProps(flinkConf2TypedProperties(FlinkOptions.flatOptions(conf)));
|
||||
|
||||
builder = builder.withSchema(getSourceSchema(conf).toString());
|
||||
|
||||
Reference in New Issue
Block a user