- Ensure that rollback instant is always created before the next commit instant.
This especially affects IncrementalPull for MOR tables since we can end up pulling in log blocks for uncommitted data - Ensure that generated commit instants are 1 second apart
This commit is contained in:
committed by
Balaji Varadarajan
parent
3a05edab01
commit
f82e58994e
@@ -135,7 +135,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
|
||||
public static SparkConf registerClasses(SparkConf conf) {
|
||||
conf.registerKryoClasses(new Class[] {HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
|
||||
conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
|
||||
return conf;
|
||||
}
|
||||
|
||||
@@ -276,7 +276,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @param bulkInsertPartitioner If specified then it will be used to partition input records before they are inserted
|
||||
* into hoodie.
|
||||
* into hoodie.
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime,
|
||||
@@ -309,7 +309,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* @param preppedRecords HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @param bulkInsertPartitioner If specified then it will be used to partition input records before they are inserted
|
||||
* into hoodie.
|
||||
* into hoodie.
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> bulkInsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String commitTime,
|
||||
@@ -953,16 +953,26 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* Provides a new commit time for a write operation (insert/update)
|
||||
*/
|
||||
public String startCommit() {
|
||||
String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
startCommitWithTime(commitTime);
|
||||
return commitTime;
|
||||
}
|
||||
|
||||
public void startCommitWithTime(String instantTime) {
|
||||
// NOTE : Need to ensure that rollback is done before a new commit is started
|
||||
if (rollbackInFlight) {
|
||||
// Only rollback inflight commit/delta-commits. Do not touch compaction commits
|
||||
rollbackInflightCommits();
|
||||
}
|
||||
String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
startCommit(commitTime);
|
||||
return commitTime;
|
||||
}
|
||||
|
||||
public void startCommitWithTime(String instantTime) {
|
||||
// NOTE : Need to ensure that rollback is done before a new commit is started
|
||||
if (rollbackInFlight) {
|
||||
// Only rollback inflight commit/delta-commits. Do not touch compaction commits
|
||||
rollbackInflightCommits();
|
||||
}
|
||||
startCommit(instantTime);
|
||||
}
|
||||
|
||||
private void startCommit(String instantTime) {
|
||||
logger.info("Generate a new instant time " + instantTime);
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
// if there are pending compactions, their instantTime must not be greater than that of this instant time
|
||||
@@ -978,7 +988,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
activeTimeline.createInflight(new HoodieInstant(true, commitActionType, instantTime));
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Schedules a new compaction instant
|
||||
*/
|
||||
|
||||
Reference in New Issue
Block a user