[HUDI-1557] Make Flink write pipeline write task scalable (#2506)
This is the #step 2 of RFC-24: https://cwiki.apache.org/confluence/display/HUDI/RFC+-+24%3A+Hoodie+Flink+Writer+Proposal This PR introduce a BucketAssigner that assigns bucket ID (partition path & fileID) for each stream record. There is no need to look up index and partition the records anymore in the following pipeline for these records, we actually decide the write target location before the write and each record computes its location when the BucketAssigner receives it, thus, the indexing is with streaming style. Computing locations for a batch of records all at a time is resource consuming so a pressure to the engine, we should avoid that in streaming system.
This commit is contained in:
@@ -30,6 +30,7 @@ import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTableVersion;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CommitUtils;
|
||||
@@ -249,7 +250,17 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
public void deletePendingInstant(String tableType, String instant) {
|
||||
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
|
||||
String commitType = CommitUtils.getCommitActionType(HoodieTableType.valueOf(tableType));
|
||||
table.getMetaClient().getActiveTimeline()
|
||||
.deletePending(new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, instant));
|
||||
HoodieActiveTimeline activeTimeline = table.getMetaClient().getActiveTimeline();
|
||||
activeTimeline.deletePending(HoodieInstant.State.INFLIGHT, commitType, instant);
|
||||
activeTimeline.deletePending(HoodieInstant.State.REQUESTED, commitType, instant);
|
||||
}
|
||||
|
||||
public void transitionRequestedToInflight(String tableType, String inFlightInstant) {
|
||||
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
String commitType = CommitUtils.getCommitActionType(HoodieTableType.valueOf(tableType));
|
||||
HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant);
|
||||
activeTimeline.transitionRequestedToInflight(requested, Option.empty(),
|
||||
config.shouldAllowMultiWriteOnSameInstant());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.FlinkHoodieIndex;
|
||||
@@ -62,47 +61,14 @@ public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends Flin
|
||||
public List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException {
|
||||
return context.map(records, record -> {
|
||||
try {
|
||||
if (mapState.contains(record.getKey())) {
|
||||
record.unseal();
|
||||
record.setCurrentLocation(mapState.get(record.getKey()));
|
||||
record.seal();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error(String.format("Tag record location failed, key = %s, %s", record.getRecordKey(), e.getMessage()));
|
||||
}
|
||||
return record;
|
||||
}, 0);
|
||||
throw new UnsupportedOperationException("No need to tag location for FlinkInMemoryStateIndex");
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException {
|
||||
return context.map(writeStatuses, writeStatus -> {
|
||||
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(record.getKey())) {
|
||||
HoodieKey key = record.getKey();
|
||||
Option<HoodieRecordLocation> newLocation = record.getNewLocation();
|
||||
if (newLocation.isPresent()) {
|
||||
try {
|
||||
mapState.put(key, newLocation.get());
|
||||
} catch (Exception e) {
|
||||
LOG.error(String.format("Update record location failed, key = %s, %s", record.getRecordKey(), e.getMessage()));
|
||||
}
|
||||
} else {
|
||||
// Delete existing index for a deleted record
|
||||
try {
|
||||
mapState.remove(key);
|
||||
} catch (Exception e) {
|
||||
LOG.error(String.format("Remove record location failed, key = %s, %s", record.getRecordKey(), e.getMessage()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return writeStatus;
|
||||
}, 0);
|
||||
throw new UnsupportedOperationException("No need to update location for FlinkInMemoryStateIndex");
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -128,6 +94,6 @@ public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends Flin
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,41 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
/**
|
||||
* Create handle factory for Flink writer, use the specified fileID directly
|
||||
* because it is unique anyway.
|
||||
*/
|
||||
public class FlinkCreateHandleFactory<T extends HoodieRecordPayload, I, K, O>
|
||||
extends CreateHandleFactory<T, I, K, O> {
|
||||
|
||||
@Override
|
||||
public HoodieWriteHandle<T, I, K, O> create(
|
||||
HoodieWriteConfig hoodieConfig, String commitTime,
|
||||
HoodieTable<T, I, K, O> hoodieTable, String partitionPath,
|
||||
String fileIdPrefix, TaskContextSupplier taskContextSupplier) {
|
||||
return new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, partitionPath,
|
||||
fileIdPrefix, taskContextSupplier);
|
||||
}
|
||||
}
|
||||
@@ -20,51 +20,52 @@ package org.apache.hudi.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.CommitUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.execution.FlinkLazyInsertIterable;
|
||||
import org.apache.hudi.io.CreateHandleFactory;
|
||||
import org.apache.hudi.io.FlinkCreateHandleFactory;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.io.HoodieSortedMergeHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.hudi.table.WorkloadStat;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* With {@code org.apache.hudi.operator.partitioner.BucketAssigner}, each hoodie record
|
||||
* is tagged with a bucket ID (partition path + fileID) in streaming way. All the records consumed by this
|
||||
* executor should be tagged with bucket IDs and belong to one data bucket.
|
||||
*
|
||||
* <p>These bucket IDs make it possible to shuffle the records first by the bucket ID
|
||||
* (see org.apache.hudi.operator.partitioner.BucketAssignerFunction), and this executor
|
||||
* only needs to handle the data buffer that belongs to one data bucket once at a time. So there is no need to
|
||||
* partition the buffer.
|
||||
*
|
||||
* <p>Computing the records batch locations all at a time is a pressure to the engine,
|
||||
* we should avoid that in streaming system.
|
||||
*/
|
||||
public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseCommitActionExecutor<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>, HoodieWriteMetadata> {
|
||||
|
||||
@@ -91,47 +92,39 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
public HoodieWriteMetadata<List<WriteStatus>> execute(List<HoodieRecord<T>> inputRecords) {
|
||||
HoodieWriteMetadata<List<WriteStatus>> result = new HoodieWriteMetadata<>();
|
||||
|
||||
WorkloadProfile profile = null;
|
||||
if (isWorkloadProfileNeeded()) {
|
||||
profile = new WorkloadProfile(buildProfile(inputRecords));
|
||||
LOG.info("Workload profile :" + profile);
|
||||
try {
|
||||
saveWorkloadProfileMetadataToInflight(profile, instantTime);
|
||||
} catch (Exception e) {
|
||||
HoodieTableMetaClient metaClient = table.getMetaClient();
|
||||
HoodieInstant inflightInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, metaClient.getCommitActionType(), instantTime);
|
||||
try {
|
||||
if (!metaClient.getFs().exists(new Path(metaClient.getMetaPath(), inflightInstant.getFileName()))) {
|
||||
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", e);
|
||||
}
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Check file exists failed");
|
||||
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final Partitioner partitioner = getPartitioner(profile);
|
||||
Map<Integer, List<HoodieRecord<T>>> partitionedRecords = partition(inputRecords, partitioner);
|
||||
|
||||
List<WriteStatus> writeStatuses = new LinkedList<>();
|
||||
partitionedRecords.forEach((partition, records) -> {
|
||||
if (WriteOperationType.isChangingRecords(operationType)) {
|
||||
handleUpsertPartition(instantTime, partition, records.iterator(), partitioner).forEachRemaining(writeStatuses::addAll);
|
||||
} else {
|
||||
handleInsertPartition(instantTime, partition, records.iterator(), partitioner).forEachRemaining(writeStatuses::addAll);
|
||||
}
|
||||
});
|
||||
updateIndex(writeStatuses, result);
|
||||
final HoodieRecord<?> record = inputRecords.get(0);
|
||||
final String partitionPath = record.getPartitionPath();
|
||||
final String fileId = record.getCurrentLocation().getFileId();
|
||||
final BucketType bucketType = record.getCurrentLocation().getInstantTime().equals("I")
|
||||
? BucketType.INSERT
|
||||
: BucketType.UPDATE;
|
||||
if (WriteOperationType.isChangingRecords(operationType)) {
|
||||
handleUpsertPartition(
|
||||
instantTime,
|
||||
partitionPath,
|
||||
fileId, bucketType,
|
||||
inputRecords.iterator())
|
||||
.forEachRemaining(writeStatuses::addAll);
|
||||
} else {
|
||||
handleUpsertPartition(
|
||||
instantTime,
|
||||
partitionPath,
|
||||
fileId,
|
||||
bucketType,
|
||||
inputRecords.iterator())
|
||||
.forEachRemaining(writeStatuses::addAll);
|
||||
}
|
||||
setUpWriteMetadata(writeStatuses, result);
|
||||
return result;
|
||||
}
|
||||
|
||||
protected void updateIndex(List<WriteStatus> writeStatuses, HoodieWriteMetadata<List<WriteStatus>> result) {
|
||||
Instant indexStartTime = Instant.now();
|
||||
// Update the index back
|
||||
List<WriteStatus> statuses = table.getIndex().updateLocation(writeStatuses, context, table);
|
||||
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
|
||||
protected void setUpWriteMetadata(
|
||||
List<WriteStatus> statuses,
|
||||
HoodieWriteMetadata<List<WriteStatus>> result) {
|
||||
// No need to update the index because the update happens before the write.
|
||||
result.setWriteStatuses(statuses);
|
||||
result.setIndexUpdateDuration(Duration.ZERO);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -139,56 +132,6 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
return table.getMetaClient().getCommitActionType();
|
||||
}
|
||||
|
||||
private Partitioner getPartitioner(WorkloadProfile profile) {
|
||||
if (WriteOperationType.isChangingRecords(operationType)) {
|
||||
return getUpsertPartitioner(profile);
|
||||
} else {
|
||||
return getInsertPartitioner(profile);
|
||||
}
|
||||
}
|
||||
|
||||
private Map<Integer, List<HoodieRecord<T>>> partition(List<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
|
||||
Map<Integer, List<Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>>> partitionedMidRecords = dedupedRecords
|
||||
.stream()
|
||||
.map(record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record))
|
||||
.collect(Collectors.groupingBy(x -> partitioner.getPartition(x._1)));
|
||||
Map<Integer, List<HoodieRecord<T>>> results = new LinkedHashMap<>();
|
||||
partitionedMidRecords.forEach((key, value) -> results.put(key, value.stream().map(x -> x._2).collect(Collectors.toList())));
|
||||
return results;
|
||||
}
|
||||
|
||||
protected Pair<HashMap<String, WorkloadStat>, WorkloadStat> buildProfile(List<HoodieRecord<T>> inputRecords) {
|
||||
HashMap<String, WorkloadStat> partitionPathStatMap = new HashMap<>();
|
||||
WorkloadStat globalStat = new WorkloadStat();
|
||||
|
||||
Map<Pair<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = inputRecords
|
||||
.stream()
|
||||
.map(record -> Pair.of(
|
||||
Pair.of(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record))
|
||||
.collect(Collectors.groupingBy(Pair::getLeft, Collectors.counting()));
|
||||
|
||||
for (Map.Entry<Pair<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts.entrySet()) {
|
||||
String partitionPath = e.getKey().getLeft();
|
||||
Long count = e.getValue();
|
||||
Option<HoodieRecordLocation> locOption = e.getKey().getRight();
|
||||
|
||||
if (!partitionPathStatMap.containsKey(partitionPath)) {
|
||||
partitionPathStatMap.put(partitionPath, new WorkloadStat());
|
||||
}
|
||||
|
||||
if (locOption.isPresent()) {
|
||||
// update
|
||||
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
|
||||
globalStat.addUpdates(locOption.get(), count);
|
||||
} else {
|
||||
// insert
|
||||
partitionPathStatMap.get(partitionPath).addInserts(count);
|
||||
globalStat.addInserts(count);
|
||||
}
|
||||
}
|
||||
return Pair.of(partitionPathStatMap, globalStat);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<List<WriteStatus>> result) {
|
||||
commit(extraMetadata, result, result.getWriteStatuses().stream().map(WriteStatus::getStat).collect(Collectors.toList()));
|
||||
@@ -228,31 +171,28 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected Iterator<List<WriteStatus>> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
|
||||
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
|
||||
BucketType btype = binfo.bucketType;
|
||||
protected Iterator<List<WriteStatus>> handleUpsertPartition(
|
||||
String instantTime,
|
||||
String partitionPath,
|
||||
String fileIdHint,
|
||||
BucketType bucketType,
|
||||
Iterator recordItr) {
|
||||
try {
|
||||
if (btype.equals(BucketType.INSERT)) {
|
||||
return handleInsert(binfo.fileIdPrefix, recordItr);
|
||||
} else if (btype.equals(BucketType.UPDATE)) {
|
||||
return handleUpdate(binfo.partitionPath, binfo.fileIdPrefix, recordItr);
|
||||
} else {
|
||||
throw new HoodieUpsertException("Unknown bucketType " + btype + " for partition :" + partition);
|
||||
switch (bucketType) {
|
||||
case INSERT:
|
||||
return handleInsert(fileIdHint, recordItr);
|
||||
case UPDATE:
|
||||
return handleUpdate(partitionPath, fileIdHint, recordItr);
|
||||
default:
|
||||
throw new HoodieUpsertException("Unknown bucketType " + bucketType + " for partition :" + partitionPath);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
|
||||
String msg = "Error upserting bucketType " + bucketType + " for partition :" + partitionPath;
|
||||
LOG.error(msg, t);
|
||||
throw new HoodieUpsertException(msg, t);
|
||||
}
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
return handleUpsertPartition(instantTime, partition, recordItr, partitioner);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr)
|
||||
@@ -293,13 +233,6 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
}
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords,
|
||||
HoodieBaseFile dataFileToBeMerged) {
|
||||
return new HoodieMergeHandle<>(config, instantTime, table, keyToNewRecords,
|
||||
partitionPath, fileId, dataFileToBeMerged, taskContextSupplier);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsert(String idPfx, Iterator<HoodieRecord<T>> recordItr)
|
||||
throws Exception {
|
||||
@@ -309,24 +242,6 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
return new FlinkLazyInsertIterable<>(recordItr, true, config, instantTime, table, idPfx,
|
||||
taskContextSupplier, new CreateHandleFactory<>());
|
||||
taskContextSupplier, new FlinkCreateHandleFactory<>());
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a partitioner to perform the upsert operation, based on the workload profile.
|
||||
*/
|
||||
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
|
||||
if (profile == null) {
|
||||
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
|
||||
}
|
||||
return new UpsertPartitioner(profile, context, table, config);
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a partitioner to perform the insert operation, based on the workload profile.
|
||||
*/
|
||||
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
|
||||
return getUpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -19,17 +19,32 @@
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Overrides the {@link #write} method to not look up index and partition the records, because
|
||||
* with {@code org.apache.hudi.operator.partitioner.BucketAssigner}, each hoodie record
|
||||
* is tagged with a bucket ID (partition path + fileID) in streaming way. The FlinkWriteHelper only hands over
|
||||
* the records to the action executor {@link BaseCommitActionExecutor} to execute.
|
||||
*
|
||||
* <p>Computing the records batch locations all at a time is a pressure to the engine,
|
||||
* we should avoid that in streaming system.
|
||||
*/
|
||||
public class FlinkWriteHelper<T extends HoodieRecordPayload,R> extends AbstractWriteHelper<T, List<HoodieRecord<T>>,
|
||||
List<HoodieKey>, List<WriteStatus>, R> {
|
||||
|
||||
@@ -44,23 +59,46 @@ public class FlinkWriteHelper<T extends HoodieRecordPayload,R> extends AbstractW
|
||||
return WriteHelperHolder.FLINK_WRITE_HELPER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> write(String instantTime, List<HoodieRecord<T>> inputRecords, HoodieEngineContext context,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table, boolean shouldCombine, int shuffleParallelism,
|
||||
BaseCommitActionExecutor<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>, R> executor, boolean performTagging) {
|
||||
try {
|
||||
Instant lookupBegin = Instant.now();
|
||||
Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now());
|
||||
|
||||
HoodieWriteMetadata<List<WriteStatus>> result = executor.execute(inputRecords);
|
||||
result.setIndexLookupDuration(indexLookupDuration);
|
||||
return result;
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieUpsertException) {
|
||||
throw (HoodieUpsertException) e;
|
||||
}
|
||||
throw new HoodieUpsertException("Failed to upsert for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> deduplicateRecords(List<HoodieRecord<T>> records,
|
||||
HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> index,
|
||||
int parallelism) {
|
||||
boolean isIndexingGlobal = index.isGlobal();
|
||||
Map<Object, List<Pair<Object, HoodieRecord<T>>>> keyedRecords = records.stream().map(record -> {
|
||||
HoodieKey hoodieKey = record.getKey();
|
||||
// If index used is global, then records are expected to differ in their partitionPath
|
||||
Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey;
|
||||
final Object key = record.getKey().getRecordKey();
|
||||
return Pair.of(key, record);
|
||||
}).collect(Collectors.groupingBy(Pair::getLeft));
|
||||
|
||||
return keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
|
||||
// we cannot allow the user to change the key or partitionPath, since that will affect
|
||||
// everything
|
||||
// so pick it from one of the records.
|
||||
HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey();
|
||||
return new HoodieRecord<T>(reducedKey, reducedData);
|
||||
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(reducedKey, reducedData);
|
||||
// reuse the location from the first record.
|
||||
hoodieRecord.setCurrentLocation(rec1.getCurrentLocation());
|
||||
return hoodieRecord;
|
||||
}).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -116,10 +116,7 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> implements Part
|
||||
private int addUpdateBucket(String partitionPath, String fileIdHint) {
|
||||
int bucket = totalBuckets;
|
||||
updateLocationToBucket.put(fileIdHint, bucket);
|
||||
BucketInfo bucketInfo = new BucketInfo();
|
||||
bucketInfo.bucketType = BucketType.UPDATE;
|
||||
bucketInfo.fileIdPrefix = fileIdHint;
|
||||
bucketInfo.partitionPath = partitionPath;
|
||||
BucketInfo bucketInfo = new BucketInfo(BucketType.UPDATE, fileIdHint, partitionPath);
|
||||
bucketInfoMap.put(totalBuckets, bucketInfo);
|
||||
totalBuckets++;
|
||||
return bucket;
|
||||
@@ -186,10 +183,7 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> implements Part
|
||||
} else {
|
||||
recordsPerBucket.add(totalUnassignedInserts - (insertBuckets - 1) * insertRecordsPerBucket);
|
||||
}
|
||||
BucketInfo bucketInfo = new BucketInfo();
|
||||
bucketInfo.bucketType = BucketType.INSERT;
|
||||
bucketInfo.partitionPath = partitionPath;
|
||||
bucketInfo.fileIdPrefix = FSUtils.createNewFileIdPfx();
|
||||
BucketInfo bucketInfo = new BucketInfo(BucketType.INSERT, FSUtils.createNewFileIdPfx(), partitionPath);
|
||||
bucketInfoMap.put(totalBuckets, bucketInfo);
|
||||
totalBuckets++;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user