[HUDI-65] commitTime rename to instantTime (#1431)
This commit is contained in:
@@ -96,23 +96,23 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload> e
|
||||
}
|
||||
|
||||
/**
|
||||
* Commit changes performed at the given commitTime marker.
|
||||
* Commit changes performed at the given instantTime marker.
|
||||
*/
|
||||
public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses) {
|
||||
return commit(commitTime, writeStatuses, Option.empty());
|
||||
public boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses) {
|
||||
return commit(instantTime, writeStatuses, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Commit changes performed at the given commitTime marker.
|
||||
* Commit changes performed at the given instantTime marker.
|
||||
*/
|
||||
public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses,
|
||||
public boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(false);
|
||||
return commit(commitTime, writeStatuses, extraMetadata, metaClient.getCommitActionType());
|
||||
return commit(instantTime, writeStatuses, extraMetadata, metaClient.getCommitActionType());
|
||||
}
|
||||
|
||||
protected JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> table,
|
||||
String commitTime) {
|
||||
String instantTime) {
|
||||
// cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future
|
||||
// RDD actions that are performed after updating the index.
|
||||
writeStatusRDD = writeStatusRDD.persist(config.getWriteStatusStorageLevel());
|
||||
@@ -121,26 +121,26 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload> e
|
||||
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, jsc, table);
|
||||
metrics.updateIndexMetrics(UPDATE_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
|
||||
// Trigger the insert and collect statuses
|
||||
commitOnAutoCommit(commitTime, statuses, table.getMetaClient().getCommitActionType());
|
||||
commitOnAutoCommit(instantTime, statuses, table.getMetaClient().getCommitActionType());
|
||||
return statuses;
|
||||
}
|
||||
|
||||
protected void commitOnAutoCommit(String commitTime, JavaRDD<WriteStatus> resultRDD, String actionType) {
|
||||
protected void commitOnAutoCommit(String instantTime, JavaRDD<WriteStatus> resultRDD, String actionType) {
|
||||
if (config.shouldAutoCommit()) {
|
||||
LOG.info("Auto commit enabled: Committing " + commitTime);
|
||||
boolean commitResult = commit(commitTime, resultRDD, Option.empty(), actionType);
|
||||
LOG.info("Auto commit enabled: Committing " + instantTime);
|
||||
boolean commitResult = commit(instantTime, resultRDD, Option.empty(), actionType);
|
||||
if (!commitResult) {
|
||||
throw new HoodieCommitException("Failed to commit " + commitTime);
|
||||
throw new HoodieCommitException("Failed to commit " + instantTime);
|
||||
}
|
||||
} else {
|
||||
LOG.info("Auto commit disabled for " + commitTime);
|
||||
LOG.info("Auto commit disabled for " + instantTime);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses,
|
||||
private boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses,
|
||||
Option<Map<String, String>> extraMetadata, String actionType) {
|
||||
|
||||
LOG.info("Commiting " + commitTime);
|
||||
LOG.info("Commiting " + instantTime);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc);
|
||||
|
||||
@@ -152,7 +152,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload> e
|
||||
updateMetadataAndRollingStats(actionType, metadata, stats);
|
||||
|
||||
// Finalize write
|
||||
finalizeWrite(table, commitTime, stats);
|
||||
finalizeWrite(table, instantTime, stats);
|
||||
|
||||
// add in extra metadata
|
||||
if (extraMetadata.isPresent()) {
|
||||
@@ -162,23 +162,23 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload> e
|
||||
metadata.setOperationType(operationType);
|
||||
|
||||
try {
|
||||
activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, commitTime),
|
||||
activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
|
||||
postCommit(metadata, commitTime, extraMetadata);
|
||||
postCommit(metadata, instantTime, extraMetadata);
|
||||
|
||||
if (writeContext != null) {
|
||||
long durationInMs = metrics.getDurationInMs(writeContext.stop());
|
||||
metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), durationInMs,
|
||||
metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(instantTime).getTime(), durationInMs,
|
||||
metadata, actionType);
|
||||
writeContext = null;
|
||||
}
|
||||
LOG.info("Committed " + commitTime);
|
||||
LOG.info("Committed " + instantTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + commitTime,
|
||||
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
|
||||
e);
|
||||
} catch (ParseException e) {
|
||||
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + commitTime
|
||||
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime
|
||||
+ "Instant time is not of valid format", e);
|
||||
}
|
||||
return true;
|
||||
|
||||
@@ -167,13 +167,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
|
||||
/**
|
||||
* Upsert a batch of new records into Hoodie table at the supplied commitTime.
|
||||
* Upsert a batch of new records into Hoodie table at the supplied instantTime.
|
||||
*
|
||||
* @param records JavaRDD of hoodieRecords to upsert
|
||||
* @param commitTime Instant time of the commit
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.UPSERT);
|
||||
setOperationType(WriteOperationType.UPSERT);
|
||||
try {
|
||||
@@ -185,34 +185,34 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
// perform index loop up to get existing location of records
|
||||
JavaRDD<HoodieRecord<T>> taggedRecords = getIndex().tagLocation(dedupedRecords, jsc, table);
|
||||
metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
|
||||
return upsertRecordsInternal(taggedRecords, commitTime, table, true);
|
||||
return upsertRecordsInternal(taggedRecords, instantTime, table, true);
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieUpsertException) {
|
||||
throw (HoodieUpsertException) e;
|
||||
}
|
||||
throw new HoodieUpsertException("Failed to upsert for commit time " + commitTime, e);
|
||||
throw new HoodieUpsertException("Failed to upsert for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Upserts the given prepared records into the Hoodie table, at the supplied commitTime.
|
||||
* Upserts the given prepared records into the Hoodie table, at the supplied instantTime.
|
||||
* <p>
|
||||
* This implementation requires that the input records are already tagged, and de-duped if needed.
|
||||
*
|
||||
* @param preppedRecords Prepared HoodieRecords to upsert
|
||||
* @param commitTime Instant time of the commit
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String commitTime) {
|
||||
public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String instantTime) {
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED);
|
||||
setOperationType(WriteOperationType.UPSERT_PREPPED);
|
||||
try {
|
||||
return upsertRecordsInternal(preppedRecords, commitTime, table, true);
|
||||
return upsertRecordsInternal(preppedRecords, instantTime, table, true);
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieUpsertException) {
|
||||
throw (HoodieUpsertException) e;
|
||||
}
|
||||
throw new HoodieUpsertException("Failed to upsert prepared records for commit time " + commitTime, e);
|
||||
throw new HoodieUpsertException("Failed to upsert prepared records for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -223,10 +223,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* alignment, as with upsert(), by profiling the workload
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Instant time of the commit
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.INSERT);
|
||||
setOperationType(WriteOperationType.INSERT);
|
||||
try {
|
||||
@@ -234,36 +234,36 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
combineOnCondition(config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism());
|
||||
|
||||
return upsertRecordsInternal(dedupedRecords, commitTime, table, false);
|
||||
return upsertRecordsInternal(dedupedRecords, instantTime, table, false);
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e);
|
||||
throw new HoodieInsertException("Failed to insert for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Inserts the given prepared records into the Hoodie table, at the supplied commitTime.
|
||||
* Inserts the given prepared records into the Hoodie table, at the supplied instantTime.
|
||||
* <p>
|
||||
* This implementation skips the index check, skips de-duping and is able to leverage benefits such as small file
|
||||
* handling/blocking alignment, as with insert(), by profiling the workload. The prepared HoodieRecords should be
|
||||
* de-duped if needed.
|
||||
*
|
||||
* @param preppedRecords HoodieRecords to insert
|
||||
* @param commitTime Instant time of the commit
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> insertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String commitTime) {
|
||||
public JavaRDD<WriteStatus> insertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String instantTime) {
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.INSERT_PREPPED);
|
||||
setOperationType(WriteOperationType.INSERT_PREPPED);
|
||||
try {
|
||||
return upsertRecordsInternal(preppedRecords, commitTime, table, false);
|
||||
return upsertRecordsInternal(preppedRecords, instantTime, table, false);
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to insert prepared records for commit time " + commitTime, e);
|
||||
throw new HoodieInsertException("Failed to insert prepared records for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -275,11 +275,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* the numbers of files with less memory compared to the {@link HoodieWriteClient#insert(JavaRDD, String)}
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Instant time of the commit
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
return bulkInsert(records, commitTime, Option.empty());
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
|
||||
return bulkInsert(records, instantTime, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -292,12 +292,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* {@link UserDefinedBulkInsertPartitioner}.
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Instant time of the commit
|
||||
* @param instantTime Instant time of the commit
|
||||
* @param bulkInsertPartitioner If specified then it will be used to partition input records before they are inserted
|
||||
* into hoodie.
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime,
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String instantTime,
|
||||
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.BULK_INSERT);
|
||||
setOperationType(WriteOperationType.BULK_INSERT);
|
||||
@@ -306,12 +306,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
combineOnCondition(config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism());
|
||||
|
||||
return bulkInsertInternal(dedupedRecords, commitTime, table, bulkInsertPartitioner);
|
||||
return bulkInsertInternal(dedupedRecords, instantTime, table, bulkInsertPartitioner);
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, e);
|
||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -326,34 +326,34 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* {@link UserDefinedBulkInsertPartitioner}.
|
||||
*
|
||||
* @param preppedRecords HoodieRecords to insert
|
||||
* @param commitTime Instant time of the commit
|
||||
* @param instantTime Instant time of the commit
|
||||
* @param bulkInsertPartitioner If specified then it will be used to partition input records before they are inserted
|
||||
* into hoodie.
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> bulkInsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String commitTime,
|
||||
public JavaRDD<WriteStatus> bulkInsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String instantTime,
|
||||
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED);
|
||||
setOperationType(WriteOperationType.BULK_INSERT_PREPPED);
|
||||
try {
|
||||
return bulkInsertInternal(preppedRecords, commitTime, table, bulkInsertPartitioner);
|
||||
return bulkInsertInternal(preppedRecords, instantTime, table, bulkInsertPartitioner);
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to bulk insert prepared records for commit time " + commitTime, e);
|
||||
throw new HoodieInsertException("Failed to bulk insert prepared records for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied commitTime {@link HoodieKey}s will be
|
||||
* Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be
|
||||
* de-duped and non existent keys will be removed before deleting.
|
||||
*
|
||||
* @param keys {@link List} of {@link HoodieKey}s to be deleted
|
||||
* @param commitTime Commit time handle
|
||||
* @param instantTime Commit time handle
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> delete(JavaRDD<HoodieKey> keys, final String commitTime) {
|
||||
public JavaRDD<WriteStatus> delete(JavaRDD<HoodieKey> keys, final String instantTime) {
|
||||
HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.DELETE);
|
||||
setOperationType(WriteOperationType.DELETE);
|
||||
try {
|
||||
@@ -370,23 +370,23 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
JavaRDD<HoodieRecord<T>> taggedValidRecords = taggedRecords.filter(HoodieRecord::isCurrentLocationKnown);
|
||||
if (!taggedValidRecords.isEmpty()) {
|
||||
metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
|
||||
return upsertRecordsInternal(taggedValidRecords, commitTime, table, true);
|
||||
return upsertRecordsInternal(taggedValidRecords, instantTime, table, true);
|
||||
} else {
|
||||
// if entire set of keys are non existent
|
||||
saveWorkloadProfileMetadataToInflight(new WorkloadProfile(jsc.emptyRDD()), table, commitTime);
|
||||
saveWorkloadProfileMetadataToInflight(new WorkloadProfile(jsc.emptyRDD()), table, instantTime);
|
||||
JavaRDD<WriteStatus> writeStatusRDD = jsc.emptyRDD();
|
||||
commitOnAutoCommit(commitTime, writeStatusRDD, table.getMetaClient().getCommitActionType());
|
||||
commitOnAutoCommit(instantTime, writeStatusRDD, table.getMetaClient().getCommitActionType());
|
||||
return writeStatusRDD;
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieUpsertException) {
|
||||
throw (HoodieUpsertException) e;
|
||||
}
|
||||
throw new HoodieUpsertException("Failed to delete for commit time " + commitTime, e);
|
||||
throw new HoodieUpsertException("Failed to delete for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
private JavaRDD<WriteStatus> bulkInsertInternal(JavaRDD<HoodieRecord<T>> dedupedRecords, String commitTime,
|
||||
private JavaRDD<WriteStatus> bulkInsertInternal(JavaRDD<HoodieRecord<T>> dedupedRecords, String instantTime,
|
||||
HoodieTable<T> table, Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
final JavaRDD<HoodieRecord<T>> repartitionedRecords;
|
||||
final int parallelism = config.getBulkInsertShuffleParallelism();
|
||||
@@ -407,13 +407,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList());
|
||||
|
||||
table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED,
|
||||
table.getMetaClient().getCommitActionType(), commitTime), Option.empty());
|
||||
table.getMetaClient().getCommitActionType(), instantTime), Option.empty());
|
||||
|
||||
JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
|
||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table, fileIDPrefixes), true)
|
||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(instantTime, config, table, fileIDPrefixes), true)
|
||||
.flatMap(List::iterator);
|
||||
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime);
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, table, instantTime);
|
||||
}
|
||||
|
||||
private JavaRDD<HoodieRecord<T>> combineOnCondition(boolean condition, JavaRDD<HoodieRecord<T>> records,
|
||||
@@ -427,7 +427,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* are unknown across batches Inserts (which are new parquet files) are rolled back based on commit time. // TODO :
|
||||
* Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata
|
||||
*/
|
||||
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, HoodieTable<T> table, String commitTime)
|
||||
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, HoodieTable<T> table, String instantTime)
|
||||
throws HoodieCommitException {
|
||||
try {
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
@@ -446,15 +446,15 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
String commitActionType = table.getMetaClient().getCommitActionType();
|
||||
HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, commitTime);
|
||||
HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime);
|
||||
activeTimeline.transitionRequestedToInflight(requested,
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieCommitException("Failed to commit " + commitTime + " unable to save inflight metadata ", io);
|
||||
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io);
|
||||
}
|
||||
}
|
||||
|
||||
private JavaRDD<WriteStatus> upsertRecordsInternal(JavaRDD<HoodieRecord<T>> preppedRecords, String commitTime,
|
||||
private JavaRDD<WriteStatus> upsertRecordsInternal(JavaRDD<HoodieRecord<T>> preppedRecords, String instantTime,
|
||||
HoodieTable<T> hoodieTable, final boolean isUpsert) {
|
||||
|
||||
// Cache the tagged records, so we don't end up computing both
|
||||
@@ -469,7 +469,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
if (hoodieTable.isWorkloadProfileNeeded()) {
|
||||
profile = new WorkloadProfile(preppedRecords);
|
||||
LOG.info("Workload profile :" + profile);
|
||||
saveWorkloadProfileMetadataToInflight(profile, hoodieTable, commitTime);
|
||||
saveWorkloadProfileMetadataToInflight(profile, hoodieTable, instantTime);
|
||||
}
|
||||
|
||||
// partition using the insert partitioner
|
||||
@@ -477,13 +477,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
JavaRDD<HoodieRecord<T>> partitionedRecords = partition(preppedRecords, partitioner);
|
||||
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords.mapPartitionsWithIndex((partition, recordItr) -> {
|
||||
if (isUpsert) {
|
||||
return hoodieTable.handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
return hoodieTable.handleUpsertPartition(instantTime, partition, recordItr, partitioner);
|
||||
} else {
|
||||
return hoodieTable.handleInsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
return hoodieTable.handleInsertPartition(instantTime, partition, recordItr, partitioner);
|
||||
}
|
||||
}, true).flatMap(List::iterator);
|
||||
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime);
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, instantTime);
|
||||
}
|
||||
|
||||
private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, WorkloadProfile profile) {
|
||||
@@ -551,7 +551,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
|
||||
/**
|
||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime will be referenced in the
|
||||
* Savepoint a specific commit. Latest version of data files as of the passed in instantTime will be referenced in the
|
||||
* savepoint and will never be cleaned. The savepointed commit will never be rolledback or archived.
|
||||
* <p>
|
||||
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be manually created and
|
||||
@@ -559,19 +559,19 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* <p>
|
||||
* Savepoint should be on a commit that could not have been cleaned.
|
||||
*
|
||||
* @param commitTime - commit that should be savepointed
|
||||
* @param instantTime - commit that should be savepointed
|
||||
* @param user - User creating the savepoint
|
||||
* @param comment - Comment for the savepoint
|
||||
* @return true if the savepoint was created successfully
|
||||
*/
|
||||
public boolean savepoint(String commitTime, String user, String comment) {
|
||||
public boolean savepoint(String instantTime, String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc);
|
||||
if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
throw new UnsupportedOperationException("Savepointing is not supported or MergeOnRead table types");
|
||||
}
|
||||
Option<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
|
||||
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
|
||||
if (!table.getCompletedCommitsTimeline().containsInstant(commitInstant)) {
|
||||
throw new HoodieSavepointException("Could not savepoint non-existing commit " + commitInstant);
|
||||
}
|
||||
@@ -589,8 +589,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
|
||||
// Cannot allow savepoint time on a commit that could have been cleaned
|
||||
ValidationUtils.checkArgument(
|
||||
HoodieTimeline.compareTimestamps(commitTime, lastCommitRetained, HoodieTimeline.GREATER_OR_EQUAL),
|
||||
"Could not savepoint commit " + commitTime + " as this is beyond the lookup window " + lastCommitRetained);
|
||||
HoodieTimeline.compareTimestamps(instantTime, lastCommitRetained, HoodieTimeline.GREATER_OR_EQUAL),
|
||||
"Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained);
|
||||
|
||||
Map<String, List<String>> latestFilesMap = jsc
|
||||
.parallelize(FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(),
|
||||
@@ -599,7 +599,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
// Scan all partitions files with this commit time
|
||||
LOG.info("Collecting latest files in partition path " + partitionPath);
|
||||
BaseFileOnlyView view = table.getBaseFileOnlyView();
|
||||
List<String> latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, commitTime)
|
||||
List<String> latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime)
|
||||
.map(HoodieBaseFile::getFileName).collect(Collectors.toList());
|
||||
return new Tuple2<>(partitionPath, latestFiles);
|
||||
}).collectAsMap();
|
||||
@@ -607,14 +607,14 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieSavepointMetadata metadata = AvroUtils.convertSavepointMetadata(user, comment, latestFilesMap);
|
||||
// Nothing to save in the savepoint
|
||||
table.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime));
|
||||
new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, instantTime));
|
||||
table.getActiveTimeline()
|
||||
.saveAsComplete(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime),
|
||||
.saveAsComplete(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, instantTime),
|
||||
AvroUtils.serializeSavepointMetadata(metadata));
|
||||
LOG.info("Savepoint " + commitTime + " created");
|
||||
LOG.info("Savepoint " + instantTime + " created");
|
||||
return true;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieSavepointException("Failed to savepoint " + commitTime, e);
|
||||
throw new HoodieSavepointException("Failed to savepoint " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -687,15 +687,15 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
// to be running. Rollback to savepoint also removes any pending compaction actions that are generated after
|
||||
// savepoint time. Allowing pending compaction to be retained is not safe as those workload could be referencing
|
||||
// file-slices that will be rolled-back as part of this operation
|
||||
HoodieTimeline commitTimeline = table.getMetaClient().getCommitsAndCompactionTimeline();
|
||||
HoodieTimeline instantTimeline = table.getMetaClient().getCommitsAndCompactionTimeline();
|
||||
|
||||
HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
|
||||
boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint);
|
||||
if (!isSavepointPresent) {
|
||||
throw new HoodieRollbackException("No savepoint for commitTime " + savepointTime);
|
||||
throw new HoodieRollbackException("No savepoint for instantTime " + savepointTime);
|
||||
}
|
||||
|
||||
List<String> commitsToRollback = commitTimeline.findInstantsAfter(savepointTime, Integer.MAX_VALUE).getInstants()
|
||||
List<String> commitsToRollback = instantTimeline.findInstantsAfter(savepointTime, Integer.MAX_VALUE).getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
LOG.info("Rolling back commits " + commitsToRollback);
|
||||
|
||||
@@ -716,11 +716,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* this commit (2) clean indexing data, (3) clean new generated parquet files. (4) Finally delete .commit or .inflight
|
||||
* file.
|
||||
*
|
||||
* @param commitTime Instant time of the commit
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return {@code true} If rollback the record changes successfully. {@code false} otherwise
|
||||
*/
|
||||
public boolean rollback(final String commitTime) throws HoodieRollbackException {
|
||||
rollbackInternal(commitTime);
|
||||
public boolean rollback(final String instantTime) throws HoodieRollbackException {
|
||||
rollbackInternal(instantTime);
|
||||
return true;
|
||||
}
|
||||
|
||||
@@ -852,9 +852,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
// Only rollback pending commit/delta-commits. Do not touch compaction commits
|
||||
rollbackPendingCommits();
|
||||
}
|
||||
String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
startCommit(commitTime);
|
||||
return commitTime;
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
startCommit(instantTime);
|
||||
return instantTime;
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -35,14 +35,14 @@ import java.util.List;
|
||||
public class BulkInsertMapFunction<T extends HoodieRecordPayload>
|
||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
|
||||
|
||||
private String commitTime;
|
||||
private String instantTime;
|
||||
private HoodieWriteConfig config;
|
||||
private HoodieTable<T> hoodieTable;
|
||||
private List<String> fileIDPrefixes;
|
||||
|
||||
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config, HoodieTable<T> hoodieTable,
|
||||
List<String> fileIDPrefixes) {
|
||||
this.commitTime = commitTime;
|
||||
public BulkInsertMapFunction(String instantTime, HoodieWriteConfig config, HoodieTable<T> hoodieTable,
|
||||
List<String> fileIDPrefixes) {
|
||||
this.instantTime = instantTime;
|
||||
this.config = config;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileIDPrefixes = fileIDPrefixes;
|
||||
@@ -50,7 +50,7 @@ public class BulkInsertMapFunction<T extends HoodieRecordPayload>
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> call(Integer partition, Iterator<HoodieRecord<T>> sortedRecordItr) {
|
||||
return new CopyOnWriteLazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable,
|
||||
return new CopyOnWriteLazyInsertIterable<>(sortedRecordItr, config, instantTime, hoodieTable,
|
||||
fileIDPrefixes.get(partition));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -46,16 +46,16 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload>
|
||||
extends LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
|
||||
|
||||
protected final HoodieWriteConfig hoodieConfig;
|
||||
protected final String commitTime;
|
||||
protected final String instantTime;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
protected final String idPrefix;
|
||||
protected int numFilesWritten;
|
||||
|
||||
public CopyOnWriteLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
||||
String commitTime, HoodieTable<T> hoodieTable, String idPrefix) {
|
||||
String instantTime, HoodieTable<T> hoodieTable, String idPrefix) {
|
||||
super(sortedRecordItr);
|
||||
this.hoodieConfig = config;
|
||||
this.commitTime = commitTime;
|
||||
this.instantTime = instantTime;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.idPrefix = idPrefix;
|
||||
this.numFilesWritten = 0;
|
||||
@@ -136,7 +136,7 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload>
|
||||
final HoodieRecord insertPayload = payload.record;
|
||||
// lazily initialize the handle, for the first time
|
||||
if (handle == null) {
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(),
|
||||
handle = new HoodieCreateHandle(hoodieConfig, instantTime, hoodieTable, insertPayload.getPartitionPath(),
|
||||
getNextFileId(idPrefix));
|
||||
}
|
||||
|
||||
@@ -147,7 +147,7 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload>
|
||||
// handle is full.
|
||||
statuses.add(handle.close());
|
||||
// Need to handle the rejected payload & open new handle
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(),
|
||||
handle = new HoodieCreateHandle(hoodieConfig, instantTime, hoodieTable, insertPayload.getPartitionPath(),
|
||||
getNextFileId(idPrefix));
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload.
|
||||
}
|
||||
|
||||
@@ -35,8 +35,8 @@ import java.util.List;
|
||||
public class MergeOnReadLazyInsertIterable<T extends HoodieRecordPayload> extends CopyOnWriteLazyInsertIterable<T> {
|
||||
|
||||
public MergeOnReadLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
||||
String commitTime, HoodieTable<T> hoodieTable, String idPfx) {
|
||||
super(sortedRecordItr, config, commitTime, hoodieTable, idPfx);
|
||||
String instantTime, HoodieTable<T> hoodieTable, String idPfx) {
|
||||
super(sortedRecordItr, config, instantTime, hoodieTable, idPfx);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -52,7 +52,7 @@ public class MergeOnReadLazyInsertIterable<T extends HoodieRecordPayload> extend
|
||||
List<WriteStatus> statuses = new ArrayList<>();
|
||||
// lazily initialize the handle, for the first time
|
||||
if (handle == null) {
|
||||
handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable,
|
||||
handle = new HoodieAppendHandle(hoodieConfig, instantTime, hoodieTable,
|
||||
insertPayload.getPartitionPath(), getNextFileId(idPrefix));
|
||||
}
|
||||
if (handle.canWrite(insertPayload)) {
|
||||
@@ -63,7 +63,7 @@ public class MergeOnReadLazyInsertIterable<T extends HoodieRecordPayload> extend
|
||||
handle.close();
|
||||
statuses.add(handle.getWriteStatus());
|
||||
// Need to handle the rejected payload & open new handle
|
||||
handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable,
|
||||
handle = new HoodieAppendHandle(hoodieConfig, instantTime, hoodieTable,
|
||||
insertPayload.getPartitionPath(), getNextFileId(idPrefix));
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload.
|
||||
}
|
||||
|
||||
@@ -98,9 +98,9 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Rollback the efffects of the commit made at commitTime.
|
||||
* Rollback the efffects of the commit made at instantTime.
|
||||
*/
|
||||
public abstract boolean rollbackCommit(String commitTime);
|
||||
public abstract boolean rollbackCommit(String instantTime);
|
||||
|
||||
/**
|
||||
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the `partitionPath`. Such an
|
||||
|
||||
@@ -94,7 +94,7 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
public boolean rollbackCommit(String instantTime) {
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
@@ -223,7 +223,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
public boolean rollbackCommit(String instantTime) {
|
||||
// Nope, don't need to do anything.
|
||||
return true;
|
||||
}
|
||||
|
||||
@@ -465,7 +465,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
public boolean rollbackCommit(String instantTime) {
|
||||
// Rollback in HbaseIndex is managed via method {@link #checkIfValidCommit()}
|
||||
return true;
|
||||
}
|
||||
|
||||
@@ -100,17 +100,17 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
|
||||
// Total number of new records inserted into the delta file
|
||||
private long insertRecordsWritten = 0;
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
super(config, commitTime, partitionPath, fileId, hoodieTable);
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable);
|
||||
writeStatus.setStat(new HoodieDeltaWriteStat());
|
||||
this.fileId = fileId;
|
||||
this.recordItr = recordItr;
|
||||
}
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath, String fileId) {
|
||||
this(config, commitTime, hoodieTable, partitionPath, fileId, null);
|
||||
this(config, instantTime, hoodieTable, partitionPath, fileId, null);
|
||||
}
|
||||
|
||||
private void init(HoodieRecord record) {
|
||||
@@ -118,7 +118,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
|
||||
// extract some information from the first record
|
||||
SliceView rtView = hoodieTable.getSliceView();
|
||||
Option<FileSlice> fileSlice = rtView.getLatestFileSlice(partitionPath, fileId);
|
||||
// Set the base commit time as the current commitTime for new inserts into log files
|
||||
// Set the base commit time as the current instantTime for new inserts into log files
|
||||
String baseInstantTime = instantTime;
|
||||
if (fileSlice.isPresent()) {
|
||||
baseInstantTime = fileSlice.get().getBaseInstantTime();
|
||||
|
||||
@@ -55,21 +55,21 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWri
|
||||
private Iterator<HoodieRecord<T>> recordIterator;
|
||||
private boolean useWriterSchema = false;
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath, String fileId) {
|
||||
super(config, commitTime, partitionPath, fileId, hoodieTable);
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
|
||||
this.path = makeNewPath(partitionPath);
|
||||
|
||||
try {
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime,
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, instantTime,
|
||||
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
createMarkerFile(partitionPath);
|
||||
this.storageWriter =
|
||||
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, hoodieTable, config, writerSchema);
|
||||
HoodieStorageWriterFactory.getStorageWriter(instantTime, path, hoodieTable, config, writerSchema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||
}
|
||||
@@ -79,9 +79,9 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWri
|
||||
/**
|
||||
* Called by the compactor code path.
|
||||
*/
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordIterator) {
|
||||
this(config, commitTime, hoodieTable, partitionPath, fileId);
|
||||
this(config, instantTime, hoodieTable, partitionPath, fileId);
|
||||
this.recordIterator = recordIterator;
|
||||
this.useWriterSchema = true;
|
||||
}
|
||||
|
||||
@@ -69,9 +69,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
||||
private long insertRecordsWritten = 0;
|
||||
private boolean useWriterSchema;
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId) {
|
||||
super(config, commitTime, partitionPath, fileId, hoodieTable);
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable);
|
||||
init(fileId, recordItr);
|
||||
init(fileId, partitionPath, hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get());
|
||||
}
|
||||
@@ -79,10 +79,10 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
||||
/**
|
||||
* Called by compactor code path.
|
||||
*/
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, String partitionPath, String fileId,
|
||||
HoodieBaseFile dataFileToBeMerged) {
|
||||
super(config, commitTime, partitionPath, fileId, hoodieTable);
|
||||
super(config, instantTime, partitionPath, fileId, hoodieTable);
|
||||
this.keyToNewRecords = keyToNewRecords;
|
||||
this.useWriterSchema = true;
|
||||
init(fileId, this.partitionPath, dataFileToBeMerged);
|
||||
|
||||
@@ -50,10 +50,10 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
private final HoodieWrapperFileSystem fs;
|
||||
private final long maxFileSize;
|
||||
private final HoodieAvroWriteSupport writeSupport;
|
||||
private final String commitTime;
|
||||
private final String instantTime;
|
||||
private final Schema schema;
|
||||
|
||||
public HoodieParquetWriter(String commitTime, Path file, HoodieParquetConfig parquetConfig, Schema schema)
|
||||
public HoodieParquetWriter(String instantTime, Path file, HoodieParquetConfig parquetConfig, Schema schema)
|
||||
throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
|
||||
@@ -70,7 +70,7 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize()
|
||||
+ Math.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.commitTime = commitTime;
|
||||
this.instantTime = instantTime;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
@@ -85,10 +85,10 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
@Override
|
||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||
String seqId =
|
||||
HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(), recordIndex.getAndIncrement());
|
||||
HoodieRecord.generateSequenceId(instantTime, TaskContext.getPartitionId(), recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(), record.getPartitionPath(),
|
||||
file.getName());
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, instantTime, seqId);
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
}
|
||||
|
||||
@@ -39,18 +39,18 @@ import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
|
||||
public class HoodieStorageWriterFactory {
|
||||
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
||||
String commitTime, Path path, HoodieTable<T> hoodieTable, HoodieWriteConfig config, Schema schema)
|
||||
String instantTime, Path path, HoodieTable<T> hoodieTable, HoodieWriteConfig config, Schema schema)
|
||||
throws IOException {
|
||||
final String name = path.getName();
|
||||
final String extension = FSUtils.isLogFile(path) ? HOODIE_LOG.getFileExtension() : FSUtils.getFileExtension(name);
|
||||
if (PARQUET.getFileExtension().equals(extension)) {
|
||||
return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable);
|
||||
return newParquetStorageWriter(instantTime, path, config, schema, hoodieTable);
|
||||
}
|
||||
throw new UnsupportedOperationException(extension + " format not supported yet.");
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
|
||||
String commitTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable)
|
||||
String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable)
|
||||
throws IOException {
|
||||
BloomFilter filter = BloomFilterFactory
|
||||
.createBloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP(),
|
||||
@@ -63,6 +63,6 @@ public class HoodieStorageWriterFactory {
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(),
|
||||
hoodieTable.getHadoopConf(), config.getParquetCompressionRatio());
|
||||
|
||||
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
|
||||
return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -245,18 +245,18 @@ public class CleanHelper<T extends HoodieRecordPayload<T>> implements Serializab
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the latest version < commitTime. This version file could still be used by queries.
|
||||
* Gets the latest version < instantTime. This version file could still be used by queries.
|
||||
*/
|
||||
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList, HoodieInstant commitTime) {
|
||||
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList, HoodieInstant instantTime) {
|
||||
for (FileSlice file : fileSliceList) {
|
||||
String fileCommitTime = file.getBaseInstantTime();
|
||||
if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
|
||||
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the
|
||||
if (HoodieTimeline.compareTimestamps(instantTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
|
||||
// fileList is sorted on the reverse, so the first commit we find <= instantTime is the
|
||||
// one we want
|
||||
return fileCommitTime;
|
||||
}
|
||||
}
|
||||
// There is no version of this file which is <= commitTime
|
||||
// There is no version of this file which is <= instantTime
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
@@ -160,7 +160,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String commitTime) {
|
||||
public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
|
||||
}
|
||||
|
||||
@@ -170,7 +170,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String partitionPath, String fileId,
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr)
|
||||
throws IOException {
|
||||
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
|
||||
@@ -179,22 +179,22 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, partitionPath, fileId, recordItr);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileId);
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(instantTime, partitionPath, fileId, recordItr);
|
||||
return handleUpdateInternal(upsertHandle, instantTime, fileId);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String partitionPath, String fileId,
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, partitionPath, fileId, keyToNewRecords, oldDataFile);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileId);
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(instantTime, partitionPath, fileId, keyToNewRecords, oldDataFile);
|
||||
return handleUpdateInternal(upsertHandle, instantTime, fileId);
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String commitTime,
|
||||
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String instantTime,
|
||||
String fileId) throws IOException {
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException(
|
||||
"Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId);
|
||||
"Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
|
||||
} else {
|
||||
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getWriterSchema());
|
||||
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
||||
@@ -221,46 +221,46 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, partitionPath, fileId);
|
||||
protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
return new HoodieMergeHandle<>(config, instantTime, this, recordItr, partitionPath, fileId);
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String partitionPath, String fileId,
|
||||
protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords,
|
||||
return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords,
|
||||
partitionPath, fileId, dataFileToBeMerged);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String idPfx, Iterator<HoodieRecord<T>> recordItr)
|
||||
public Iterator<List<WriteStatus>> handleInsert(String instantTime, String idPfx, Iterator<HoodieRecord<T>> recordItr)
|
||||
throws Exception {
|
||||
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
|
||||
if (!recordItr.hasNext()) {
|
||||
LOG.info("Empty partition");
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
return new CopyOnWriteLazyInsertIterable<>(recordItr, config, commitTime, this, idPfx);
|
||||
return new CopyOnWriteLazyInsertIterable<>(recordItr, config, instantTime, this, idPfx);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String partitionPath, String fileId,
|
||||
public Iterator<List<WriteStatus>> handleInsert(String instantTime, String partitionPath, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
HoodieCreateHandle createHandle =
|
||||
new HoodieCreateHandle(config, commitTime, this, partitionPath, fileId, recordItr);
|
||||
new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordItr);
|
||||
createHandle.write();
|
||||
return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime, Integer partition, Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
public 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;
|
||||
try {
|
||||
if (btype.equals(BucketType.INSERT)) {
|
||||
return handleInsert(commitTime, binfo.fileIdPrefix, recordItr);
|
||||
return handleInsert(instantTime, binfo.fileIdPrefix, recordItr);
|
||||
} else if (btype.equals(BucketType.UPDATE)) {
|
||||
return handleUpdate(commitTime, binfo.partitionPath, binfo.fileIdPrefix, recordItr);
|
||||
return handleUpdate(instantTime, binfo.partitionPath, binfo.fileIdPrefix, recordItr);
|
||||
} else {
|
||||
throw new HoodieUpsertException("Unknown bucketType " + btype + " for partition :" + partition);
|
||||
}
|
||||
@@ -272,9 +272,9 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsertPartition(String commitTime, Integer partition, Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
return handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
public Iterator<List<WriteStatus>> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
return handleUpsertPartition(instantTime, partition, recordItr, partitioner);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -98,16 +98,16 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String partitionPath,
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath,
|
||||
String fileId, Iterator<HoodieRecord<T>> recordItr)
|
||||
throws IOException {
|
||||
LOG.info("Merging updates for commit " + commitTime + " for file " + fileId);
|
||||
LOG.info("Merging updates for commit " + instantTime + " for file " + fileId);
|
||||
|
||||
if (!index.canIndexLogFiles() && mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
|
||||
LOG.info("Small file corrections for updates for commit " + commitTime + " for file " + fileId);
|
||||
return super.handleUpdate(commitTime, partitionPath, fileId, recordItr);
|
||||
LOG.info("Small file corrections for updates for commit " + instantTime + " for file " + fileId);
|
||||
return super.handleUpdate(instantTime, partitionPath, fileId, recordItr);
|
||||
} else {
|
||||
HoodieAppendHandle<T> appendHandle = new HoodieAppendHandle<>(config, commitTime, this,
|
||||
HoodieAppendHandle<T> appendHandle = new HoodieAppendHandle<>(config, instantTime, this,
|
||||
partitionPath, fileId, recordItr);
|
||||
appendHandle.doAppend();
|
||||
appendHandle.close();
|
||||
@@ -116,13 +116,13 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String idPfx, Iterator<HoodieRecord<T>> recordItr)
|
||||
public Iterator<List<WriteStatus>> handleInsert(String instantTime, String idPfx, Iterator<HoodieRecord<T>> recordItr)
|
||||
throws Exception {
|
||||
// If canIndexLogFiles, write inserts to log files else write inserts to parquet files
|
||||
if (index.canIndexLogFiles()) {
|
||||
return new MergeOnReadLazyInsertIterable<>(recordItr, config, commitTime, this, idPfx);
|
||||
return new MergeOnReadLazyInsertIterable<>(recordItr, config, instantTime, this, idPfx);
|
||||
} else {
|
||||
return super.handleInsert(commitTime, idPfx, recordItr);
|
||||
return super.handleInsert(instantTime, idPfx, recordItr);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -248,13 +248,13 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
/**
|
||||
* Perform the ultimate IO for a given upserted (RDD) partition.
|
||||
*/
|
||||
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime, Integer partition,
|
||||
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String instantTime, Integer partition,
|
||||
Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
/**
|
||||
* Perform the ultimate IO for a given inserted (RDD) partition.
|
||||
*/
|
||||
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime, Integer partition,
|
||||
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String instantTime, Integer partition,
|
||||
Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
/**
|
||||
|
||||
@@ -99,12 +99,12 @@ public class HoodieMergeOnReadTableCompactor implements HoodieCompactor {
|
||||
}
|
||||
|
||||
private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
|
||||
HoodieWriteConfig config, CompactionOperation operation, String commitTime) throws IOException {
|
||||
HoodieWriteConfig config, CompactionOperation operation, String instantTime) throws IOException {
|
||||
FileSystem fs = metaClient.getFs();
|
||||
|
||||
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
LOG.info("Compacting base " + operation.getDataFileName() + " with delta files " + operation.getDeltaFileNames()
|
||||
+ " for commit " + commitTime);
|
||||
+ " for commit " + instantTime);
|
||||
// TODO - FIX THIS
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file
|
||||
// (failure recover).
|
||||
@@ -136,11 +136,11 @@ public class HoodieMergeOnReadTableCompactor implements HoodieCompactor {
|
||||
// If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a
|
||||
// new base parquet file.
|
||||
if (oldDataFileOpt.isPresent()) {
|
||||
result = hoodieCopyOnWriteTable.handleUpdate(commitTime, operation.getPartitionPath(),
|
||||
result = hoodieCopyOnWriteTable.handleUpdate(instantTime, operation.getPartitionPath(),
|
||||
operation.getFileId(), scanner.getRecords(),
|
||||
oldDataFileOpt.get());
|
||||
} else {
|
||||
result = hoodieCopyOnWriteTable.handleInsert(commitTime, operation.getPartitionPath(), operation.getFileId(),
|
||||
result = hoodieCopyOnWriteTable.handleInsert(instantTime, operation.getPartitionPath(), operation.getFileId(),
|
||||
scanner.iterator());
|
||||
}
|
||||
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
||||
|
||||
Reference in New Issue
Block a user