[HUDI-858] Allow multiple operations to be executed within a single commit (#1633)
This commit is contained in:
committed by
GitHub
parent
2600d2de8d
commit
e6f3bf10cf
@@ -99,6 +99,20 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
|
||||
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7;
|
||||
|
||||
/**
|
||||
* HUDI-858 : There are users who had been directly using RDD APIs and have relied on a behavior in 0.4.x to allow
|
||||
* multiple write operations (upsert/buk-insert/...) to be executed within a single commit.
|
||||
*
|
||||
* Given Hudi commit protocol, these are generally unsafe operations and user need to handle failure scenarios. It
|
||||
* only works with COW table. Hudi 0.5.x had stopped this behavior.
|
||||
*
|
||||
* Given the importance of supporting such cases for the user's migration to 0.5.x, we are proposing a safety flag
|
||||
* (disabled by default) which will allow this old behavior.
|
||||
*/
|
||||
private static final String ALLOW_MULTI_WRITE_ON_SAME_INSTANT =
|
||||
"_.hoodie.allow.multi.write.on.same.instant";
|
||||
private static final String DEFAULT_ALLOW_MULTI_WRITE_ON_SAME_INSTANT = "false";
|
||||
|
||||
private ConsistencyGuardConfig consistencyGuardConfig;
|
||||
|
||||
// Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled
|
||||
@@ -194,6 +208,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_DELETE_PROP));
|
||||
}
|
||||
|
||||
public boolean shouldAllowMultiWriteOnSameInstant() {
|
||||
return Boolean.parseBoolean(props.getProperty(ALLOW_MULTI_WRITE_ON_SAME_INSTANT));
|
||||
}
|
||||
|
||||
public String getWriteStatusClassName() {
|
||||
return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP);
|
||||
}
|
||||
@@ -723,6 +741,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAllowMultiWriteOnSameInstant(boolean allow) {
|
||||
props.setProperty(ALLOW_MULTI_WRITE_ON_SAME_INSTANT, String.valueOf(allow));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteConfig build() {
|
||||
// Check for mandatory properties
|
||||
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM, DEFAULT_PARALLELISM);
|
||||
@@ -738,6 +761,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
DEFAULT_COMBINE_BEFORE_UPSERT);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_DELETE_PROP), COMBINE_BEFORE_DELETE_PROP,
|
||||
DEFAULT_COMBINE_BEFORE_DELETE);
|
||||
setDefaultOnCondition(props, !props.containsKey(ALLOW_MULTI_WRITE_ON_SAME_INSTANT),
|
||||
ALLOW_MULTI_WRITE_ON_SAME_INSTANT, DEFAULT_ALLOW_MULTI_WRITE_ON_SAME_INSTANT);
|
||||
setDefaultOnCondition(props, !props.containsKey(WRITE_STATUS_STORAGE_LEVEL), WRITE_STATUS_STORAGE_LEVEL,
|
||||
DEFAULT_WRITE_STATUS_STORAGE_LEVEL);
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP), HOODIE_AUTO_COMMIT_PROP,
|
||||
@@ -778,7 +803,6 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
// Ensure Layout Version is good
|
||||
new TimelineLayoutVersion(Integer.parseInt(layoutVersion));
|
||||
|
||||
|
||||
// Build WriteConfig at the end
|
||||
HoodieWriteConfig config = new HoodieWriteConfig(props);
|
||||
Objects.requireNonNull(config.getBasePath());
|
||||
|
||||
@@ -136,7 +136,8 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
String commitActionType = table.getMetaClient().getCommitActionType();
|
||||
HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime);
|
||||
activeTimeline.transitionRequestedToInflight(requested,
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)),
|
||||
config.shouldAllowMultiWriteOnSameInstant());
|
||||
} catch (IOException io) {
|
||||
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io);
|
||||
}
|
||||
|
||||
@@ -73,7 +73,8 @@ public class BulkInsertHelper<T extends HoodieRecordPayload<T>> {
|
||||
IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList());
|
||||
|
||||
table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED,
|
||||
table.getMetaClient().getCommitActionType(), instantTime), Option.empty());
|
||||
table.getMetaClient().getCommitActionType(), instantTime), Option.empty(),
|
||||
config.shouldAllowMultiWriteOnSameInstant());
|
||||
|
||||
JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
|
||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(instantTime, config, table, fileIDPrefixes), true)
|
||||
|
||||
Reference in New Issue
Block a user