1
0

[HUDI-1788] Insert overwrite (table) for Flink writer (#2808)

Supports `INSERT OVERWRITE` and `INSERT OVERWRITE TABLE` for Flink
writer.
This commit is contained in:
Danny Chan
2021-04-14 10:23:37 +08:00
committed by GitHub
parent 65844a8d29
commit ab4a7b0b4a
23 changed files with 523 additions and 93 deletions

View File

@@ -22,6 +22,8 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.client.utils.TransactionUtils;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
@@ -139,10 +141,8 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
table.validateUpsertSchema();
preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient());
final HoodieRecord<T> record = records.get(0);
final boolean isDelta = table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ);
final HoodieWriteHandle<?, ?, ?, ?> writeHandle = getOrCreateWriteHandle(record, isDelta, getConfig(),
instantTime, table, record.getPartitionPath(), records.listIterator());
final HoodieWriteHandle<?, ?, ?, ?> writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(),
instantTime, table, records.listIterator());
HoodieWriteMetadata<List<WriteStatus>> result = ((HoodieFlinkTable<T>) table).upsert(context, writeHandle, instantTime, records);
if (result.getIndexLookupDuration().isPresent()) {
metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
@@ -162,10 +162,8 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
table.validateUpsertSchema();
preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient());
// create the write handle if not exists
final HoodieRecord<T> record = records.get(0);
final boolean isDelta = table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ);
final HoodieWriteHandle<?, ?, ?, ?> writeHandle = getOrCreateWriteHandle(record, isDelta, getConfig(),
instantTime, table, record.getPartitionPath(), records.listIterator());
final HoodieWriteHandle<?, ?, ?, ?> writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(),
instantTime, table, records.listIterator());
HoodieWriteMetadata<List<WriteStatus>> result = ((HoodieFlinkTable<T>) table).insert(context, writeHandle, instantTime, records);
if (result.getIndexLookupDuration().isPresent()) {
metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
@@ -173,6 +171,45 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
return postWrite(result, instantTime, table);
}
/**
* Removes all existing records from the partitions affected and inserts the given HoodieRecords, into the table.
*
* @param records HoodieRecords to insert
* @param instantTime Instant time of the commit
* @return list of WriteStatus to inspect errors and counts
*/
public List<WriteStatus> insertOverwrite(
List<HoodieRecord<T>> records, final String instantTime) {
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime);
table.validateInsertSchema();
preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE, table.getMetaClient());
// create the write handle if not exists
final HoodieWriteHandle<?, ?, ?, ?> writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(),
instantTime, table, records.listIterator());
HoodieWriteMetadata result = ((HoodieFlinkTable<T>) table).insertOverwrite(context, writeHandle, instantTime, records);
return postWrite(result, instantTime, table);
}
/**
* Removes all existing records of the Hoodie table and inserts the given HoodieRecords, into the table.
*
* @param records HoodieRecords to insert
* @param instantTime Instant time of the commit
* @return list of WriteStatus to inspect errors and counts
*/
public List<WriteStatus> insertOverwriteTable(
List<HoodieRecord<T>> records, final String instantTime) {
HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE_TABLE, instantTime);
table.validateInsertSchema();
preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, table.getMetaClient());
// create the write handle if not exists
final HoodieWriteHandle<?, ?, ?, ?> writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(),
instantTime, table, records.listIterator());
HoodieWriteMetadata result = ((HoodieFlinkTable<T>) table).insertOverwriteTable(context, writeHandle, instantTime, records);
return postWrite(result, instantTime, table);
}
@Override
public List<WriteStatus> insertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime) {
throw new HoodieNotSupportedException("InsertPrepped operation is not supported yet");
@@ -353,27 +390,25 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
* Get or create a new write handle in order to reuse the file handles.
*
* @param record The first record in the bucket
* @param isDelta Whether the table is in MOR mode
* @param config Write config
* @param instantTime The instant time
* @param table The table
* @param partitionPath Partition path
* @param recordItr Record iterator
* @return Existing write handle or create a new one
*/
private HoodieWriteHandle<?, ?, ?, ?> getOrCreateWriteHandle(
HoodieRecord<T> record,
boolean isDelta,
HoodieWriteConfig config,
String instantTime,
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
String partitionPath,
Iterator<HoodieRecord<T>> recordItr) {
final HoodieRecordLocation loc = record.getCurrentLocation();
final String fileID = loc.getFileId();
if (bucketToHandles.containsKey(fileID)) {
return bucketToHandles.get(fileID);
}
final String partitionPath = record.getPartitionPath();
final boolean isDelta = table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ);
final HoodieWriteHandle<?, ?, ?, ?> writeHandle;
if (isDelta) {
writeHandle = new FlinkAppendHandle<>(config, instantTime, table, partitionPath, fileID, recordItr,
@@ -409,19 +444,23 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
.collect(Collectors.toList());
}
public String getInflightAndRequestedInstant(String tableType) {
final String commitType = CommitUtils.getCommitActionType(HoodieTableType.valueOf(tableType));
public String getLastPendingInstant(HoodieTableType tableType) {
final String actionType = CommitUtils.getCommitActionType(tableType);
return getLastPendingInstant(actionType);
}
public String getLastPendingInstant(String actionType) {
HoodieTimeline unCompletedTimeline = getHoodieTable().getMetaClient().getCommitsTimeline().filterInflightsAndRequested();
return unCompletedTimeline.getInstants()
.filter(x -> x.getAction().equals(commitType))
.filter(x -> x.getAction().equals(actionType))
.map(HoodieInstant::getTimestamp)
.collect(Collectors.toList()).stream()
.max(Comparator.naturalOrder())
.orElse(null);
}
public String getLastCompletedInstant(String tableType) {
final String commitType = CommitUtils.getCommitActionType(HoodieTableType.valueOf(tableType));
public String getLastCompletedInstant(HoodieTableType tableType) {
final String commitType = CommitUtils.getCommitActionType(tableType);
HoodieTimeline completedTimeline = getHoodieTable().getMetaClient().getCommitsTimeline().filterCompletedInstants();
return completedTimeline.getInstants()
.filter(x -> x.getAction().equals(commitType))
@@ -431,32 +470,49 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
.orElse(null);
}
public void deletePendingInstant(String tableType, String instant) {
HoodieFlinkTable<T> table = getHoodieTable();
String commitType = CommitUtils.getCommitActionType(HoodieTableType.valueOf(tableType));
HoodieActiveTimeline activeTimeline = table.getMetaClient().getActiveTimeline();
activeTimeline.deletePendingIfExists(HoodieInstant.State.INFLIGHT, commitType, instant);
activeTimeline.deletePendingIfExists(HoodieInstant.State.REQUESTED, commitType, instant);
}
public void transitionRequestedToInflight(String tableType, String inFlightInstant) {
public void transitionRequestedToInflight(String commitType, String inFlightInstant) {
HoodieFlinkTable<T> table = getHoodieTable();
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());
}
public void rollbackInflightCompaction(HoodieInstant inflightInstant) {
HoodieFlinkTable<T> table = getHoodieTable();
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
rollbackInflightCompaction(inflightInstant, table);
}
}
public HoodieFlinkTable<T> getHoodieTable() {
return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
}
public Map<String, List<String>> getPartitionToReplacedFileIds(
WriteOperationType writeOperationType,
List<WriteStatus> writeStatuses) {
HoodieFlinkTable<T> table = getHoodieTable();
switch (writeOperationType) {
case INSERT_OVERWRITE:
return writeStatuses.stream().map(status -> status.getStat().getPartitionPath()).distinct()
.collect(
Collectors.toMap(
partition -> partition,
partitionPath -> getAllExistingFileIds(table, partitionPath)));
case INSERT_OVERWRITE_TABLE:
Map<String, List<String>> partitionToExistingFileIds = new HashMap<>();
List<String> partitionPaths =
FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath());
if (partitionPaths != null && partitionPaths.size() > 0) {
context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions");
partitionToExistingFileIds = partitionPaths.stream().parallel()
.collect(
Collectors.toMap(
partition -> partition,
partition -> getAllExistingFileIds(table, partition)));
}
return partitionToExistingFileIds;
default:
throw new AssertionError();
}
}
private List<String> getAllExistingFileIds(HoodieFlinkTable<T> table, String partitionPath) {
// because new commit is not complete. it is safe to mark all existing file Ids as old files
return table.getSliceView().getLatestFileSlices(partitionPath).map(FileSlice::getFileId).distinct().collect(Collectors.toList());
}
}

View File

@@ -125,4 +125,36 @@ public interface ExplicitWriteHandleTable<T extends HoodieRecordPayload> {
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
String instantTime,
List<HoodieRecord<T>> preppedRecords);
/**
* Replaces all the existing records and inserts the specified new records into Hoodie table at the supplied instantTime,
* for the partition paths contained in input records.
*
* @param context HoodieEngineContext
* @param writeHandle The write handle
* @param instantTime Instant time for the replace action
* @param records input records
* @return HoodieWriteMetadata
*/
HoodieWriteMetadata<List<WriteStatus>> insertOverwrite(
HoodieEngineContext context,
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
String instantTime,
List<HoodieRecord<T>> records);
/**
* Deletes all the existing records of the Hoodie table and inserts the specified new records into Hoodie table at the supplied instantTime,
* for the partition paths contained in input records.
*
* @param context HoodieEngineContext
* @param writeHandle The write handle
* @param instantTime Instant time for the replace action
* @param records input records
* @return HoodieWriteMetadata
*/
HoodieWriteMetadata<List<WriteStatus>> insertOverwriteTable(
HoodieEngineContext context,
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
String instantTime,
List<HoodieRecord<T>> records);
}

View File

@@ -47,6 +47,8 @@ import org.apache.hudi.table.action.clean.FlinkCleanActionExecutor;
import org.apache.hudi.table.action.clean.FlinkScheduleCleanActionExecutor;
import org.apache.hudi.table.action.commit.FlinkDeleteCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkInsertCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkInsertOverwriteCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkInsertOverwriteTableCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkInsertPreppedCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkMergeHelper;
import org.apache.hudi.table.action.commit.FlinkUpsertCommitActionExecutor;
@@ -181,6 +183,24 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload> extends
return new FlinkInsertPreppedCommitActionExecutor<>(context, writeHandle, config, this, instantTime, preppedRecords).execute();
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> insertOverwrite(
HoodieEngineContext context,
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
String instantTime,
List<HoodieRecord<T>> records) {
return new FlinkInsertOverwriteCommitActionExecutor(context, writeHandle, config, this, instantTime, records).execute();
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> insertOverwriteTable(
HoodieEngineContext context,
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
String instantTime,
List<HoodieRecord<T>> records) {
return new FlinkInsertOverwriteTableCommitActionExecutor(context, writeHandle, config, this, instantTime, records).execute();
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> upsert(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> records) {
throw new HoodieNotSupportedException("This method should not be invoked");
@@ -229,12 +249,12 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload> extends
@Override
public HoodieWriteMetadata<List<WriteStatus>> insertOverwrite(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> records) {
throw new HoodieNotSupportedException("InsertOverWrite is not supported yet");
throw new HoodieNotSupportedException("This method should not be invoked");
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> insertOverwriteTable(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> records) {
throw new HoodieNotSupportedException("insertOverwriteTable is not supported yet");
throw new HoodieNotSupportedException("This method should not be invoked");
}
@Override

View File

@@ -0,0 +1,69 @@
/*
* 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.table.action.commit;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.engine.HoodieEngineContext;
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.table.timeline.HoodieTimeline;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.HoodieWriteHandle;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import java.util.List;
public class FlinkInsertOverwriteCommitActionExecutor<T extends HoodieRecordPayload<T>>
extends BaseFlinkCommitActionExecutor<T> {
protected List<HoodieRecord<T>> inputRecords;
public FlinkInsertOverwriteCommitActionExecutor(HoodieEngineContext context,
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
HoodieWriteConfig config,
HoodieTable table,
String instantTime,
List<HoodieRecord<T>> inputRecords) {
this(context, writeHandle, config, table, instantTime, inputRecords, WriteOperationType.INSERT_OVERWRITE);
}
public FlinkInsertOverwriteCommitActionExecutor(HoodieEngineContext context,
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
HoodieWriteConfig config,
HoodieTable table,
String instantTime,
List<HoodieRecord<T>> inputRecords,
WriteOperationType writeOperationType) {
super(context, writeHandle, config, table, instantTime, writeOperationType);
this.inputRecords = inputRecords;
}
@Override
protected String getCommitActionType() {
return HoodieTimeline.REPLACE_COMMIT_ACTION;
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> execute() {
return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table,
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false);
}
}

View File

@@ -0,0 +1,50 @@
/*
* 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.table.action.commit;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.engine.HoodieEngineContext;
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.config.HoodieWriteConfig;
import org.apache.hudi.io.HoodieWriteHandle;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import java.util.List;
public class FlinkInsertOverwriteTableCommitActionExecutor<T extends HoodieRecordPayload<T>>
extends FlinkInsertOverwriteCommitActionExecutor<T> {
public FlinkInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context,
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
HoodieWriteConfig config,
HoodieTable table,
String instantTime,
List<HoodieRecord<T>> inputRecords) {
super(context, writeHandle, config, table, instantTime, inputRecords, WriteOperationType.INSERT_OVERWRITE_TABLE);
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> execute() {
return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table,
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false);
}
}