This reverts commit 371526789d.
This commit is contained in:
@@ -19,5 +19,5 @@
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
public enum BucketType {
|
||||
UPDATE, INSERT, APPEND_ONLY
|
||||
UPDATE, INSERT
|
||||
}
|
||||
|
||||
@@ -56,7 +56,6 @@ import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.HoodieTimelineArchiveLog;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.commit.BucketType;
|
||||
import org.apache.hudi.table.action.compact.FlinkCompactHelpers;
|
||||
import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
|
||||
import org.apache.hudi.util.FlinkClientUtil;
|
||||
@@ -409,12 +408,6 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
final HoodieRecordLocation loc = record.getCurrentLocation();
|
||||
final String fileID = loc.getFileId();
|
||||
final String partitionPath = record.getPartitionPath();
|
||||
// append only mode always use FlinkCreateHandle
|
||||
if (loc.getInstantTime().equals(BucketType.APPEND_ONLY.name())) {
|
||||
return new FlinkCreateHandle<>(config, instantTime, table, partitionPath,
|
||||
fileID, table.getTaskContextSupplier());
|
||||
}
|
||||
|
||||
if (bucketToHandles.containsKey(fileID)) {
|
||||
MiniBatchHandle lastHandle = (MiniBatchHandle) bucketToHandles.get(fileID);
|
||||
if (lastHandle.shouldReplace()) {
|
||||
@@ -431,8 +424,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
if (isDelta) {
|
||||
writeHandle = new FlinkAppendHandle<>(config, instantTime, table, partitionPath, fileID, recordItr,
|
||||
table.getTaskContextSupplier());
|
||||
} else if (loc.getInstantTime().equals(BucketType.INSERT.name()) || loc.getInstantTime().equals(BucketType.APPEND_ONLY.name())) {
|
||||
// use the same handle for insert bucket
|
||||
} else if (loc.getInstantTime().equals("I")) {
|
||||
writeHandle = new FlinkCreateHandle<>(config, instantTime, table, partitionPath,
|
||||
fileID, table.getTaskContextSupplier());
|
||||
} else {
|
||||
|
||||
@@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
import org.apache.hudi.table.action.commit.BucketType;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
@@ -81,7 +80,7 @@ public class FlinkAppendHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
@Override
|
||||
protected boolean isUpdateRecord(HoodieRecord<T> hoodieRecord) {
|
||||
return hoodieRecord.getCurrentLocation() != null
|
||||
&& hoodieRecord.getCurrentLocation().getInstantTime().equals(BucketType.UPDATE.name());
|
||||
&& hoodieRecord.getCurrentLocation().getInstantTime().equals("U");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -30,7 +30,6 @@ import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -71,17 +70,6 @@ public class FlinkCreateHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void createMarkerFile(String partitionPath, String dataFileName) {
|
||||
// In some rare cases, the task was pulled up again with same write file name,
|
||||
// for e.g, reuse the small log files from last commit instant.
|
||||
|
||||
// Just skip the marker file creation if it already exists, the new data would append to
|
||||
// the file directly.
|
||||
MarkerFiles markerFiles = new MarkerFiles(hoodieTable, instantTime);
|
||||
markerFiles.createIfNotExists(partitionPath, dataFileName, getIOType());
|
||||
}
|
||||
|
||||
/**
|
||||
* The flink checkpoints start in sequence and asynchronously, when one write task finish the checkpoint(A)
|
||||
* (thus the fs view got the written data files some of which may be invalid),
|
||||
|
||||
@@ -102,7 +102,9 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
final HoodieRecord<?> record = inputRecords.get(0);
|
||||
final String partitionPath = record.getPartitionPath();
|
||||
final String fileId = record.getCurrentLocation().getFileId();
|
||||
final BucketType bucketType = BucketType.valueOf(record.getCurrentLocation().getInstantTime());
|
||||
final BucketType bucketType = record.getCurrentLocation().getInstantTime().equals("I")
|
||||
? BucketType.INSERT
|
||||
: BucketType.UPDATE;
|
||||
handleUpsertPartition(
|
||||
instantTime,
|
||||
partitionPath,
|
||||
@@ -183,7 +185,6 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
} else {
|
||||
switch (bucketType) {
|
||||
case INSERT:
|
||||
case APPEND_ONLY:
|
||||
return handleInsert(fileIdHint, recordItr);
|
||||
case UPDATE:
|
||||
return handleUpdate(partitionPath, fileIdHint, recordItr);
|
||||
|
||||
Reference in New Issue
Block a user