[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;
|
||||
|
||||
@@ -212,13 +212,13 @@ public class TestHoodieClientBase extends HoodieClientTestHarness {
|
||||
* Ensure records have location field set.
|
||||
*
|
||||
* @param taggedRecords Tagged Records
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param instantTime Commit Timestamp
|
||||
*/
|
||||
protected void checkTaggedRecords(List<HoodieRecord> taggedRecords, String commitTime) {
|
||||
protected void checkTaggedRecords(List<HoodieRecord> taggedRecords, String instantTime) {
|
||||
for (HoodieRecord rec : taggedRecords) {
|
||||
assertTrue("Record " + rec + " found with no location.", rec.isCurrentLocationKnown());
|
||||
assertEquals("All records should have commit time " + commitTime + ", since updates were made",
|
||||
rec.getCurrentLocation().getInstantTime(), commitTime);
|
||||
assertEquals("All records should have commit time " + instantTime + ", since updates were made",
|
||||
rec.getCurrentLocation().getInstantTime(), instantTime);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -129,7 +129,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
*/
|
||||
@Test
|
||||
public void testAutoCommitOnBulkInsertPrepped() throws Exception {
|
||||
testAutoCommit((writeClient, recordRDD, commitTime) -> writeClient.bulkInsertPreppedRecords(recordRDD, commitTime,
|
||||
testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.bulkInsertPreppedRecords(recordRDD, instantTime,
|
||||
Option.empty()), true);
|
||||
}
|
||||
|
||||
@@ -357,9 +357,9 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
|
||||
final List<HoodieRecord> recordsInFirstBatch = new ArrayList<>();
|
||||
Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
|
||||
(String commitTime, Integer numRecordsInThisCommit) -> {
|
||||
List<HoodieRecord> fewRecordsForInsert = dataGen.generateInserts(commitTime, 200);
|
||||
List<HoodieRecord> fewRecordsForDelete = dataGen.generateDeletes(commitTime, 100);
|
||||
(String instantTime, Integer numRecordsInThisCommit) -> {
|
||||
List<HoodieRecord> fewRecordsForInsert = dataGen.generateInserts(instantTime, 200);
|
||||
List<HoodieRecord> fewRecordsForDelete = dataGen.generateDeletes(instantTime, 100);
|
||||
|
||||
recordsInFirstBatch.addAll(fewRecordsForInsert);
|
||||
recordsInFirstBatch.addAll(fewRecordsForDelete);
|
||||
@@ -376,7 +376,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
newCommitTime = "004";
|
||||
final List<HoodieRecord> recordsInSecondBatch = new ArrayList<>();
|
||||
|
||||
recordGenFunction = (String commitTime, Integer numRecordsInThisCommit) -> {
|
||||
recordGenFunction = (String instantTime, Integer numRecordsInThisCommit) -> {
|
||||
List<HoodieRecord> fewRecordsForDelete = recordsInFirstBatch.subList(0, 50);
|
||||
List<HoodieRecord> fewRecordsForUpdate = recordsInFirstBatch.subList(50, 100);
|
||||
recordsInSecondBatch.addAll(dataGen.generateDeletesFromExistingRecords(fewRecordsForDelete));
|
||||
@@ -704,18 +704,18 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
testDeletes(client, updateBatch3.getRight(), 10, file1, "007", 140, keysSoFar);
|
||||
}
|
||||
|
||||
private Pair<Set<String>, List<HoodieRecord>> testUpdates(String commitTime, HoodieWriteClient client,
|
||||
private Pair<Set<String>, List<HoodieRecord>> testUpdates(String instantTime, HoodieWriteClient client,
|
||||
int sizeToInsertAndUpdate, int expectedTotalRecords)
|
||||
throws IOException {
|
||||
client.startCommitWithTime(commitTime);
|
||||
List<HoodieRecord> inserts = dataGen.generateInserts(commitTime, sizeToInsertAndUpdate);
|
||||
client.startCommitWithTime(instantTime);
|
||||
List<HoodieRecord> inserts = dataGen.generateInserts(instantTime, sizeToInsertAndUpdate);
|
||||
Set<String> keys = HoodieClientTestUtils.getRecordKeys(inserts);
|
||||
List<HoodieRecord> insertsAndUpdates = new ArrayList<>();
|
||||
insertsAndUpdates.addAll(inserts);
|
||||
insertsAndUpdates.addAll(dataGen.generateUpdates(commitTime, inserts));
|
||||
insertsAndUpdates.addAll(dataGen.generateUpdates(instantTime, inserts));
|
||||
|
||||
JavaRDD<HoodieRecord> insertAndUpdatesRDD = jsc.parallelize(insertsAndUpdates, 1);
|
||||
List<WriteStatus> statuses = client.upsert(insertAndUpdatesRDD, commitTime).collect();
|
||||
List<WriteStatus> statuses = client.upsert(insertAndUpdatesRDD, instantTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// Check the entire dataset has all records still
|
||||
@@ -729,13 +729,13 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
}
|
||||
|
||||
private void testDeletes(HoodieWriteClient client, List<HoodieRecord> previousRecords, int sizeToDelete,
|
||||
String existingFile, String commitTime, int exepctedRecords, List<String> keys) {
|
||||
client.startCommitWithTime(commitTime);
|
||||
String existingFile, String instantTime, int exepctedRecords, List<String> keys) {
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
List<HoodieKey> hoodieKeysToDelete = HoodieClientTestUtils
|
||||
.getKeysToDelete(HoodieClientTestUtils.getHoodieKeys(previousRecords), sizeToDelete);
|
||||
JavaRDD<HoodieKey> deleteKeys = jsc.parallelize(hoodieKeysToDelete, 1);
|
||||
List<WriteStatus> statuses = client.delete(deleteKeys, commitTime).collect();
|
||||
List<WriteStatus> statuses = client.delete(deleteKeys, instantTime).collect();
|
||||
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
@@ -757,7 +757,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile);
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertTrue("key expected to be part of " + commitTime, keys.contains(recordKey));
|
||||
assertTrue("key expected to be part of " + instantTime, keys.contains(recordKey));
|
||||
assertFalse("Key deleted", hoodieKeysToDelete.contains(recordKey));
|
||||
}
|
||||
}
|
||||
@@ -802,21 +802,21 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
String commitTime = "000";
|
||||
client.startCommitWithTime(commitTime);
|
||||
String instantTime = "000";
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(instantTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, instantTime);
|
||||
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result));
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
HoodieTestUtils.doesCommitExist(basePath, instantTime));
|
||||
|
||||
// Get parquet file paths from commit metadata
|
||||
String actionType = metaClient.getCommitActionType();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime);
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, actionType, instantTime);
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(commitInstant).get(), HoodieCommitMetadata.class);
|
||||
@@ -824,7 +824,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
Collection<String> commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values();
|
||||
|
||||
// Read from commit file
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, instantTime);
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
String everything = FileIOUtils.readAsUTFString(inputStream);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything, HoodieCommitMetadata.class);
|
||||
@@ -848,20 +848,20 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
|
||||
String commitTime = "000";
|
||||
client.startCommitWithTime(commitTime);
|
||||
String instantTime = "000";
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(instantTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, instantTime);
|
||||
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result));
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
HoodieTestUtils.doesCommitExist(basePath, instantTime));
|
||||
|
||||
// Read from commit file
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, instantTime);
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
String everything = FileIOUtils.readAsUTFString(inputStream);
|
||||
HoodieCommitMetadata metadata =
|
||||
@@ -879,19 +879,19 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
Assert.assertEquals(inserts, 200);
|
||||
|
||||
// Update + Inserts such that they just expand file1
|
||||
commitTime = "001";
|
||||
client.startCommitWithTime(commitTime);
|
||||
instantTime = "001";
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
records = dataGen.generateUpdates(commitTime, records);
|
||||
records = dataGen.generateUpdates(instantTime, records);
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
result = client.upsert(writeRecords, commitTime);
|
||||
result = client.upsert(writeRecords, instantTime);
|
||||
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result));
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, result));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
HoodieTestUtils.doesCommitExist(basePath, instantTime));
|
||||
|
||||
// Read from commit file
|
||||
filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
filename = HoodieTestUtils.getCommitFilePath(basePath, instantTime);
|
||||
inputStream = new FileInputStream(filename);
|
||||
everything = FileIOUtils.readAsUTFString(inputStream);
|
||||
metadata = HoodieCommitMetadata.fromJsonString(everything.toString(), HoodieCommitMetadata.class);
|
||||
@@ -918,37 +918,37 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
@Test
|
||||
public void testConsistencyCheckDuringFinalize() throws Exception {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
String commitTime = "000";
|
||||
String instantTime = "000";
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
Pair<Path, JavaRDD<WriteStatus>> result = testConsistencyCheck(metaClient, commitTime);
|
||||
Pair<Path, JavaRDD<WriteStatus>> result = testConsistencyCheck(metaClient, instantTime);
|
||||
|
||||
// Delete orphan marker and commit should succeed
|
||||
metaClient.getFs().delete(result.getKey(), false);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, result.getRight()));
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, result.getRight()));
|
||||
assertTrue("After explicit commit, commit file should be created",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
HoodieTestUtils.doesCommitExist(basePath, instantTime));
|
||||
// Marker directory must be removed
|
||||
assertFalse(metaClient.getFs().exists(new Path(metaClient.getMarkerFolderPath(commitTime))));
|
||||
assertFalse(metaClient.getFs().exists(new Path(metaClient.getMarkerFolderPath(instantTime))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollbackAfterConsistencyCheckFailure() throws Exception {
|
||||
String commitTime = "000";
|
||||
String instantTime = "000";
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
testConsistencyCheck(metaClient, commitTime);
|
||||
testConsistencyCheck(metaClient, instantTime);
|
||||
|
||||
// Rollback of this commit should succeed
|
||||
client.rollback(commitTime);
|
||||
client.rollback(instantTime);
|
||||
assertFalse("After explicit rollback, commit file should not be present",
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
HoodieTestUtils.doesCommitExist(basePath, instantTime));
|
||||
// Marker directory must be removed after rollback
|
||||
assertFalse(metaClient.getFs().exists(new Path(metaClient.getMarkerFolderPath(commitTime))));
|
||||
assertFalse(metaClient.getFs().exists(new Path(metaClient.getMarkerFolderPath(instantTime))));
|
||||
}
|
||||
|
||||
private Pair<Path, JavaRDD<WriteStatus>> testConsistencyCheck(HoodieTableMetaClient metaClient, String commitTime)
|
||||
private Pair<Path, JavaRDD<WriteStatus>> testConsistencyCheck(HoodieTableMetaClient metaClient, String instantTime)
|
||||
throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true)
|
||||
@@ -956,24 +956,24 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
.build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
|
||||
client.startCommitWithTime(commitTime);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(dataGen.generateInserts(commitTime, 200), 1);
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
|
||||
client.startCommitWithTime(instantTime);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(dataGen.generateInserts(instantTime, 200), 1);
|
||||
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, instantTime);
|
||||
result.collect();
|
||||
|
||||
// Create a dummy marker file to simulate the case that a marker file was created without data file.
|
||||
// This should fail the commit
|
||||
String partitionPath = Arrays
|
||||
.stream(fs.globStatus(new Path(String.format("%s/*/*/*/*", metaClient.getMarkerFolderPath(commitTime))),
|
||||
.stream(fs.globStatus(new Path(String.format("%s/*/*/*/*", metaClient.getMarkerFolderPath(instantTime))),
|
||||
path -> path.toString().endsWith(HoodieTableMetaClient.MARKER_EXTN)))
|
||||
.limit(1).map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0);
|
||||
Path markerFilePath = new Path(String.format("%s/%s", partitionPath,
|
||||
FSUtils.makeMarkerFile(commitTime, "1-0-1", UUID.randomUUID().toString())));
|
||||
FSUtils.makeMarkerFile(instantTime, "1-0-1", UUID.randomUUID().toString())));
|
||||
metaClient.getFs().create(markerFilePath);
|
||||
LOG.info("Created a dummy marker path=" + markerFilePath);
|
||||
|
||||
try {
|
||||
client.commit(commitTime, result);
|
||||
client.commit(instantTime, result);
|
||||
fail("Commit should fail due to consistency check");
|
||||
} catch (HoodieCommitException cme) {
|
||||
assertTrue(cme.getCause() instanceof HoodieIOException);
|
||||
|
||||
@@ -74,8 +74,8 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
||||
@Test
|
||||
public void testReadFilterExistAfterBulkInsertPrepped() throws Exception {
|
||||
testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(),
|
||||
(writeClient, recordRDD, commitTime) -> {
|
||||
return writeClient.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty());
|
||||
(writeClient, recordRDD, instantTime) -> {
|
||||
return writeClient.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty());
|
||||
});
|
||||
}
|
||||
|
||||
@@ -178,8 +178,8 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
||||
@Test
|
||||
public void testTagLocationAfterBulkInsertPrepped() throws Exception {
|
||||
testTagLocation(
|
||||
getConfigBuilder().withBulkInsertParallelism(1).build(), (writeClient, recordRDD, commitTime) -> writeClient
|
||||
.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()),
|
||||
getConfigBuilder().withBulkInsertParallelism(1).build(), (writeClient, recordRDD, instantTime) -> writeClient
|
||||
.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()),
|
||||
HoodieWriteClient::upsertPreppedRecords, true);
|
||||
}
|
||||
|
||||
|
||||
@@ -115,30 +115,30 @@ public class HoodieClientTestUtils {
|
||||
return toReturn;
|
||||
}
|
||||
|
||||
private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException {
|
||||
private static void fakeMetaFile(String basePath, String instantTime, String suffix) throws IOException {
|
||||
String parentPath = basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME;
|
||||
new File(parentPath).mkdirs();
|
||||
new File(parentPath + "/" + commitTime + suffix).createNewFile();
|
||||
new File(parentPath + "/" + instantTime + suffix).createNewFile();
|
||||
}
|
||||
|
||||
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
|
||||
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_EXTENSION);
|
||||
public static void fakeCommitFile(String basePath, String instantTime) throws IOException {
|
||||
fakeMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION);
|
||||
}
|
||||
|
||||
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
|
||||
fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION);
|
||||
public static void fakeInFlightFile(String basePath, String instantTime) throws IOException {
|
||||
fakeMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_EXTENSION);
|
||||
}
|
||||
|
||||
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId)
|
||||
public static void fakeDataFile(String basePath, String partitionPath, String instantTime, String fileId)
|
||||
throws Exception {
|
||||
fakeDataFile(basePath, partitionPath, commitTime, fileId, 0);
|
||||
fakeDataFile(basePath, partitionPath, instantTime, fileId, 0);
|
||||
}
|
||||
|
||||
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId, long length)
|
||||
public static void fakeDataFile(String basePath, String partitionPath, String instantTime, String fileId, long length)
|
||||
throws Exception {
|
||||
String parentPath = String.format("%s/%s", basePath, partitionPath);
|
||||
new File(parentPath).mkdirs();
|
||||
String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, "1-0-1", fileId));
|
||||
String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(instantTime, "1-0-1", fileId));
|
||||
new File(path).createNewFile();
|
||||
new RandomAccessFile(path, "rw").setLength(length);
|
||||
}
|
||||
@@ -161,19 +161,19 @@ public class HoodieClientTestUtils {
|
||||
}
|
||||
|
||||
public static Dataset<Row> readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
|
||||
String commitTime) {
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
String instantTime) {
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
|
||||
if (!commitTimeline.containsInstant(commitInstant)) {
|
||||
throw new HoodieException("No commit exists at " + commitTime);
|
||||
throw new HoodieException("No commit exists at " + instantTime);
|
||||
}
|
||||
try {
|
||||
HashMap<String, String> paths =
|
||||
getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant));
|
||||
LOG.info("Path :" + paths.values());
|
||||
return sqlContext.read().parquet(paths.values().toArray(new String[paths.size()]))
|
||||
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime));
|
||||
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, instantTime));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error reading commit " + commitTime, e);
|
||||
throw new HoodieException("Error reading commit " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -225,16 +225,16 @@ public class HoodieClientTestUtils {
|
||||
}
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
String commitTime = FSUtils.getCommitTime(filename);
|
||||
String instantTime = FSUtils.getCommitTime(filename);
|
||||
HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
|
||||
HoodieTestUtils.getDefaultHadoopConf(), Double.valueOf(HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO));
|
||||
HoodieParquetWriter writer =
|
||||
new HoodieParquetWriter(commitTime, new Path(basePath + "/" + partitionPath + "/" + filename), config, schema);
|
||||
new HoodieParquetWriter(instantTime, new Path(basePath + "/" + partitionPath + "/" + filename), config, schema);
|
||||
int seqId = 1;
|
||||
for (HoodieRecord record : records) {
|
||||
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
|
||||
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++);
|
||||
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, instantTime, "" + seqId++);
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), filename);
|
||||
writer.writeAvro(record.getRecordKey(), avroRecord);
|
||||
filter.add(record.getRecordKey());
|
||||
@@ -243,7 +243,7 @@ public class HoodieClientTestUtils {
|
||||
|
||||
if (createCommitTime) {
|
||||
HoodieTestUtils.createMetadataFolder(basePath);
|
||||
HoodieTestUtils.createCommitFiles(basePath, commitTime);
|
||||
HoodieTestUtils.createCommitFiles(basePath, instantTime);
|
||||
}
|
||||
return filename;
|
||||
}
|
||||
@@ -251,10 +251,10 @@ public class HoodieClientTestUtils {
|
||||
public static String writeParquetFile(String basePath, String partitionPath, List<HoodieRecord> records,
|
||||
Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException {
|
||||
Thread.sleep(1000);
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
String instantTime = HoodieTestUtils.makeNewCommitTime();
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
String filename = FSUtils.makeDataFileName(commitTime, "1-0-1", fileId);
|
||||
HoodieTestUtils.createCommitFiles(basePath, commitTime);
|
||||
String filename = FSUtils.makeDataFileName(instantTime, "1-0-1", fileId);
|
||||
HoodieTestUtils.createCommitFiles(basePath, instantTime);
|
||||
return HoodieClientTestUtils.writeParquetFile(basePath, partitionPath, filename, records, schema, filter,
|
||||
createCommitTime);
|
||||
}
|
||||
|
||||
@@ -129,12 +129,12 @@ public class HoodieTestDataGenerator {
|
||||
* retaining the key if optionally provided.
|
||||
*
|
||||
* @param key Hoodie key.
|
||||
* @param commitTime Commit time to use.
|
||||
* @param instantTime Instant time to use.
|
||||
* @return Raw paylaod of a test record.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) throws IOException {
|
||||
return generateRandomValue(key, commitTime, false);
|
||||
public static TestRawTripPayload generateRandomValue(HoodieKey key, String instantTime) throws IOException {
|
||||
return generateRandomValue(key, instantTime, false);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -142,15 +142,15 @@ public class HoodieTestDataGenerator {
|
||||
* retaining the key if optionally provided.
|
||||
*
|
||||
* @param key Hoodie key.
|
||||
* @param commitTime Commit time to use.
|
||||
* @param instantTime Commit time to use.
|
||||
* @param isFlattened whether the schema of the record should be flattened.
|
||||
* @return Raw paylaod of a test record.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static TestRawTripPayload generateRandomValue(
|
||||
HoodieKey key, String commitTime, boolean isFlattened) throws IOException {
|
||||
HoodieKey key, String instantTime, boolean isFlattened) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(
|
||||
key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0,
|
||||
key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0.0,
|
||||
false, isFlattened);
|
||||
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
|
||||
}
|
||||
@@ -158,8 +158,8 @@ public class HoodieTestDataGenerator {
|
||||
/**
|
||||
* Generates a new avro record of the above schema format for a delete.
|
||||
*/
|
||||
public static TestRawTripPayload generateRandomDeleteValue(HoodieKey key, String commitTime) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0,
|
||||
public static TestRawTripPayload generateRandomDeleteValue(HoodieKey key, String instantTime) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0.0,
|
||||
true, false);
|
||||
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
|
||||
}
|
||||
@@ -167,8 +167,8 @@ public class HoodieTestDataGenerator {
|
||||
/**
|
||||
* Generates a new avro record of the above schema format, retaining the key if optionally provided.
|
||||
*/
|
||||
public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String commitTime) {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0);
|
||||
public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String instantTime) {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0.0);
|
||||
return new HoodieAvroPayload(Option.of(rec));
|
||||
}
|
||||
|
||||
@@ -208,9 +208,9 @@ public class HoodieTestDataGenerator {
|
||||
return rec;
|
||||
}
|
||||
|
||||
public static void createCommitFile(String basePath, String commitTime, Configuration configuration) {
|
||||
Arrays.asList(HoodieTimeline.makeCommitFileName(commitTime), HoodieTimeline.makeInflightCommitFileName(commitTime),
|
||||
HoodieTimeline.makeRequestedCommitFileName(commitTime))
|
||||
public static void createCommitFile(String basePath, String instantTime, Configuration configuration) {
|
||||
Arrays.asList(HoodieTimeline.makeCommitFileName(instantTime), HoodieTimeline.makeInflightCommitFileName(instantTime),
|
||||
HoodieTimeline.makeRequestedCommitFileName(instantTime))
|
||||
.forEach(f -> {
|
||||
Path commitFile = new Path(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f);
|
||||
@@ -235,10 +235,10 @@ public class HoodieTestDataGenerator {
|
||||
});
|
||||
}
|
||||
|
||||
public static void createCompactionRequestedFile(String basePath, String commitTime, Configuration configuration)
|
||||
public static void createCompactionRequestedFile(String basePath, String instantTime, Configuration configuration)
|
||||
throws IOException {
|
||||
Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeRequestedCompactionFileName(commitTime));
|
||||
+ HoodieTimeline.makeRequestedCompactionFileName(instantTime));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
os.close();
|
||||
@@ -256,10 +256,10 @@ public class HoodieTestDataGenerator {
|
||||
}
|
||||
}
|
||||
|
||||
public static void createSavepointFile(String basePath, String commitTime, Configuration configuration)
|
||||
public static void createSavepointFile(String basePath, String instantTime, Configuration configuration)
|
||||
throws IOException {
|
||||
Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeSavePointFileName(commitTime));
|
||||
+ HoodieTimeline.makeSavePointFileName(instantTime));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
try (FSDataOutputStream os = fs.create(commitFile, true)) {
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
@@ -272,28 +272,28 @@ public class HoodieTestDataGenerator {
|
||||
* Generates new inserts with nested schema, uniformly across the partition paths above.
|
||||
* It also updates the list of existing keys.
|
||||
*/
|
||||
public List<HoodieRecord> generateInserts(String commitTime, Integer n) {
|
||||
return generateInserts(commitTime, n, false);
|
||||
public List<HoodieRecord> generateInserts(String instantTime, Integer n) {
|
||||
return generateInserts(instantTime, n, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates new inserts, uniformly across the partition paths above.
|
||||
* It also updates the list of existing keys.
|
||||
*
|
||||
* @param commitTime Commit time to use.
|
||||
* @param instantTime Commit time to use.
|
||||
* @param n Number of records.
|
||||
* @param isFlattened whether the schema of the generated record is flattened
|
||||
* @return List of {@link HoodieRecord}s
|
||||
*/
|
||||
public List<HoodieRecord> generateInserts(String commitTime, Integer n, boolean isFlattened) {
|
||||
return generateInsertsStream(commitTime, n, isFlattened).collect(Collectors.toList());
|
||||
public List<HoodieRecord> generateInserts(String instantTime, Integer n, boolean isFlattened) {
|
||||
return generateInsertsStream(instantTime, n, isFlattened).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
|
||||
*/
|
||||
public Stream<HoodieRecord> generateInsertsStream(
|
||||
String commitTime, Integer n, boolean isFlattened) {
|
||||
String instantTime, Integer n, boolean isFlattened) {
|
||||
int currSize = getNumExistingKeys();
|
||||
|
||||
return IntStream.range(0, n).boxed().map(i -> {
|
||||
@@ -305,30 +305,30 @@ public class HoodieTestDataGenerator {
|
||||
existingKeys.put(currSize + i, kp);
|
||||
numExistingKeys++;
|
||||
try {
|
||||
return new HoodieRecord(key, generateRandomValue(key, commitTime, isFlattened));
|
||||
return new HoodieRecord(key, generateRandomValue(key, instantTime, isFlattened));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public List<HoodieRecord> generateSameKeyInserts(String commitTime, List<HoodieRecord> origin) throws IOException {
|
||||
public List<HoodieRecord> generateSameKeyInserts(String instantTime, List<HoodieRecord> origin) throws IOException {
|
||||
List<HoodieRecord> copy = new ArrayList<>();
|
||||
for (HoodieRecord r : origin) {
|
||||
HoodieKey key = r.getKey();
|
||||
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
|
||||
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, instantTime));
|
||||
copy.add(record);
|
||||
}
|
||||
return copy;
|
||||
}
|
||||
|
||||
public List<HoodieRecord> generateInsertsWithHoodieAvroPayload(String commitTime, int limit) {
|
||||
public List<HoodieRecord> generateInsertsWithHoodieAvroPayload(String instantTime, int limit) {
|
||||
List<HoodieRecord> inserts = new ArrayList<>();
|
||||
int currSize = getNumExistingKeys();
|
||||
for (int i = 0; i < limit; i++) {
|
||||
String partitionPath = partitionPaths[RAND.nextInt(partitionPaths.length)];
|
||||
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
|
||||
HoodieRecord record = new HoodieRecord(key, generateAvroPayload(key, commitTime));
|
||||
HoodieRecord record = new HoodieRecord(key, generateAvroPayload(key, instantTime));
|
||||
inserts.add(record);
|
||||
|
||||
KeyPartition kp = new KeyPartition();
|
||||
@@ -340,17 +340,17 @@ public class HoodieTestDataGenerator {
|
||||
return inserts;
|
||||
}
|
||||
|
||||
public List<HoodieRecord> generateUpdatesWithHoodieAvroPayload(String commitTime, List<HoodieRecord> baseRecords) {
|
||||
public List<HoodieRecord> generateUpdatesWithHoodieAvroPayload(String instantTime, List<HoodieRecord> baseRecords) {
|
||||
List<HoodieRecord> updates = new ArrayList<>();
|
||||
for (HoodieRecord baseRecord : baseRecords) {
|
||||
HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateAvroPayload(baseRecord.getKey(), commitTime));
|
||||
HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateAvroPayload(baseRecord.getKey(), instantTime));
|
||||
updates.add(record);
|
||||
}
|
||||
return updates;
|
||||
}
|
||||
|
||||
public List<HoodieRecord> generateDeletes(String commitTime, Integer n) throws IOException {
|
||||
List<HoodieRecord> inserts = generateInserts(commitTime, n);
|
||||
public List<HoodieRecord> generateDeletes(String instantTime, Integer n) throws IOException {
|
||||
List<HoodieRecord> inserts = generateInserts(instantTime, n);
|
||||
return generateDeletesFromExistingRecords(inserts);
|
||||
}
|
||||
|
||||
@@ -374,20 +374,20 @@ public class HoodieTestDataGenerator {
|
||||
return new HoodieRecord(key, payload);
|
||||
}
|
||||
|
||||
public HoodieRecord generateUpdateRecord(HoodieKey key, String commitTime) throws IOException {
|
||||
return new HoodieRecord(key, generateRandomValue(key, commitTime));
|
||||
public HoodieRecord generateUpdateRecord(HoodieKey key, String instantTime) throws IOException {
|
||||
return new HoodieRecord(key, generateRandomValue(key, instantTime));
|
||||
}
|
||||
|
||||
public List<HoodieRecord> generateUpdates(String commitTime, List<HoodieRecord> baseRecords) throws IOException {
|
||||
public List<HoodieRecord> generateUpdates(String instantTime, List<HoodieRecord> baseRecords) throws IOException {
|
||||
List<HoodieRecord> updates = new ArrayList<>();
|
||||
for (HoodieRecord baseRecord : baseRecords) {
|
||||
HoodieRecord record = generateUpdateRecord(baseRecord.getKey(), commitTime);
|
||||
HoodieRecord record = generateUpdateRecord(baseRecord.getKey(), instantTime);
|
||||
updates.add(record);
|
||||
}
|
||||
return updates;
|
||||
}
|
||||
|
||||
public List<HoodieRecord> generateUpdatesWithDiffPartition(String commitTime, List<HoodieRecord> baseRecords)
|
||||
public List<HoodieRecord> generateUpdatesWithDiffPartition(String instantTime, List<HoodieRecord> baseRecords)
|
||||
throws IOException {
|
||||
List<HoodieRecord> updates = new ArrayList<>();
|
||||
for (HoodieRecord baseRecord : baseRecords) {
|
||||
@@ -399,7 +399,7 @@ public class HoodieTestDataGenerator {
|
||||
newPartition = partitionPaths[0];
|
||||
}
|
||||
HoodieKey key = new HoodieKey(baseRecord.getRecordKey(), newPartition);
|
||||
HoodieRecord record = generateUpdateRecord(key, commitTime);
|
||||
HoodieRecord record = generateUpdateRecord(key, instantTime);
|
||||
updates.add(record);
|
||||
}
|
||||
return updates;
|
||||
@@ -409,15 +409,15 @@ public class HoodieTestDataGenerator {
|
||||
* Generates new updates, randomly distributed across the keys above. There can be duplicates within the returned
|
||||
* list
|
||||
*
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param instantTime Instant Timestamp
|
||||
* @param n Number of updates (including dups)
|
||||
* @return list of hoodie record updates
|
||||
*/
|
||||
public List<HoodieRecord> generateUpdates(String commitTime, Integer n) throws IOException {
|
||||
public List<HoodieRecord> generateUpdates(String instantTime, Integer n) throws IOException {
|
||||
List<HoodieRecord> updates = new ArrayList<>();
|
||||
for (int i = 0; i < n; i++) {
|
||||
KeyPartition kp = existingKeys.get(RAND.nextInt(numExistingKeys - 1));
|
||||
HoodieRecord record = generateUpdateRecord(kp.key, commitTime);
|
||||
HoodieRecord record = generateUpdateRecord(kp.key, instantTime);
|
||||
updates.add(record);
|
||||
}
|
||||
return updates;
|
||||
@@ -426,12 +426,12 @@ public class HoodieTestDataGenerator {
|
||||
/**
|
||||
* Generates deduped updates of keys previously inserted, randomly distributed across the keys above.
|
||||
*
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param instantTime Instant Timestamp
|
||||
* @param n Number of unique records
|
||||
* @return list of hoodie record updates
|
||||
*/
|
||||
public List<HoodieRecord> generateUniqueUpdates(String commitTime, Integer n) {
|
||||
return generateUniqueUpdatesStream(commitTime, n).collect(Collectors.toList());
|
||||
public List<HoodieRecord> generateUniqueUpdates(String instantTime, Integer n) {
|
||||
return generateUniqueUpdatesStream(instantTime, n).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -447,11 +447,11 @@ public class HoodieTestDataGenerator {
|
||||
/**
|
||||
* Generates deduped updates of keys previously inserted, randomly distributed across the keys above.
|
||||
*
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param instantTime Commit Timestamp
|
||||
* @param n Number of unique records
|
||||
* @return stream of hoodie record updates
|
||||
*/
|
||||
public Stream<HoodieRecord> generateUniqueUpdatesStream(String commitTime, Integer n) {
|
||||
public Stream<HoodieRecord> generateUniqueUpdatesStream(String instantTime, Integer n) {
|
||||
final Set<KeyPartition> used = new HashSet<>();
|
||||
if (n > numExistingKeys) {
|
||||
throw new IllegalArgumentException("Requested unique updates is greater than number of available keys");
|
||||
@@ -467,7 +467,7 @@ public class HoodieTestDataGenerator {
|
||||
}
|
||||
used.add(kp);
|
||||
try {
|
||||
return new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
|
||||
return new HoodieRecord(kp.key, generateRandomValue(kp.key, instantTime));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
@@ -505,11 +505,11 @@ public class HoodieTestDataGenerator {
|
||||
/**
|
||||
* Generates deduped delete records previously inserted, randomly distributed across the keys above.
|
||||
*
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param instantTime Commit Timestamp
|
||||
* @param n Number of unique records
|
||||
* @return stream of hoodie records for delete
|
||||
*/
|
||||
public Stream<HoodieRecord> generateUniqueDeleteRecordStream(String commitTime, Integer n) {
|
||||
public Stream<HoodieRecord> generateUniqueDeleteRecordStream(String instantTime, Integer n) {
|
||||
final Set<KeyPartition> used = new HashSet<>();
|
||||
if (n > numExistingKeys) {
|
||||
throw new IllegalArgumentException("Requested unique deletes is greater than number of available keys");
|
||||
@@ -528,7 +528,7 @@ public class HoodieTestDataGenerator {
|
||||
numExistingKeys--;
|
||||
used.add(kp);
|
||||
try {
|
||||
result.add(new HoodieRecord(kp.key, generateRandomDeleteValue(kp.key, commitTime)));
|
||||
result.add(new HoodieRecord(kp.key, generateRandomDeleteValue(kp.key, instantTime)));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
|
||||
@@ -43,7 +43,7 @@ import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
|
||||
private final String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
private final String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
@@ -58,7 +58,7 @@ public class TestBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
@Test
|
||||
public void testExecutor() {
|
||||
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(commitTime, 100);
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, 100);
|
||||
|
||||
HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
|
||||
when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024);
|
||||
|
||||
@@ -59,7 +59,7 @@ import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
|
||||
private final String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
private final String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
@@ -79,7 +79,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
@Test(timeout = 60000)
|
||||
public void testRecordReading() throws Exception {
|
||||
final int numRecords = 128;
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(commitTime, numRecords);
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
// Produce
|
||||
@@ -126,7 +126,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
Map<String, Tuple2<Integer, Integer>> keyToProducerAndIndexMap = new HashMap<>();
|
||||
|
||||
for (int i = 0; i < numProducers; i++) {
|
||||
List<HoodieRecord> pRecs = dataGen.generateInserts(commitTime, numRecords);
|
||||
List<HoodieRecord> pRecs = dataGen.generateInserts(instantTime, numRecords);
|
||||
int j = 0;
|
||||
for (HoodieRecord r : pRecs) {
|
||||
Assert.assertTrue(!keyToProducerAndIndexMap.containsKey(r.getRecordKey()));
|
||||
@@ -209,7 +209,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
@Test(timeout = 60000)
|
||||
public void testMemoryLimitForBuffering() throws Exception {
|
||||
final int numRecords = 128;
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(commitTime, numRecords);
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
// maximum number of records to keep in memory.
|
||||
final int recordLimit = 5;
|
||||
final SizeEstimator<HoodieInsertValueGenResult<HoodieRecord>> sizeEstimator = new DefaultSizeEstimator<>();
|
||||
@@ -258,7 +258,7 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
@Test(timeout = 60000)
|
||||
public void testException() throws Exception {
|
||||
final int numRecords = 256;
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(commitTime, numRecords);
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
final SizeEstimator<Tuple2<HoodieRecord, Option<IndexedRecord>>> sizeEstimator = new DefaultSizeEstimator<>();
|
||||
// queue memory limit
|
||||
HoodieInsertValueGenResult<HoodieRecord> payload =
|
||||
|
||||
@@ -125,7 +125,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
public boolean rollbackCommit(String instantTime) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
@@ -40,18 +40,18 @@ public class TestHoodieStorageWriterFactory extends TestHoodieClientBase {
|
||||
@Test
|
||||
public void testGetStorageWriter() throws IOException {
|
||||
// parquet file format.
|
||||
final String commitTime = "100";
|
||||
final String instantTime = "100";
|
||||
final Path parquetPath = new Path(basePath + "/partition/path/f1_1-0-1_000.parquet");
|
||||
final HoodieWriteConfig cfg = getConfig();
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieStorageWriter<IndexedRecord> parquetWriter = HoodieStorageWriterFactory.getStorageWriter(commitTime,
|
||||
HoodieStorageWriter<IndexedRecord> parquetWriter = HoodieStorageWriterFactory.getStorageWriter(instantTime,
|
||||
parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA);
|
||||
Assert.assertTrue(parquetWriter instanceof HoodieParquetWriter);
|
||||
|
||||
// other file format exception.
|
||||
final Path logPath = new Path(basePath + "/partition/path/f.b51192a8-574b-4a85-b246-bcfec03ac8bf_100.log.2_1-0-1");
|
||||
try {
|
||||
HoodieStorageWriter<IndexedRecord> logWriter = HoodieStorageWriterFactory.getStorageWriter(commitTime, logPath,
|
||||
HoodieStorageWriter<IndexedRecord> logWriter = HoodieStorageWriterFactory.getStorageWriter(instantTime, logPath,
|
||||
table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA);
|
||||
fail("should fail since log storage writer is not supported yet.");
|
||||
} catch (Exception e) {
|
||||
|
||||
@@ -131,9 +131,9 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, client.getConfig(), jsc);
|
||||
|
||||
assertFalse(table.getCompletedCommitsTimeline().empty());
|
||||
String commitTime = table.getCompletedCommitsTimeline().getInstants().findFirst().get().getTimestamp();
|
||||
String instantTime = table.getCompletedCommitsTimeline().getInstants().findFirst().get().getTimestamp();
|
||||
assertFalse(table.getCompletedCleanTimeline().empty());
|
||||
assertEquals("The clean instant should be the same as the commit instant", commitTime,
|
||||
assertEquals("The clean instant should be the same as the commit instant", instantTime,
|
||||
table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp());
|
||||
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
||||
@@ -173,7 +173,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
@Test
|
||||
public void testBulkInsertPreppedAndCleanByVersions() throws Exception {
|
||||
testInsertAndCleanByVersions(
|
||||
(client, recordRDD, commitTime) -> client.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()),
|
||||
(client, recordRDD, instantTime) -> client.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()),
|
||||
HoodieWriteClient::upsertPreppedRecords, true);
|
||||
}
|
||||
|
||||
@@ -325,7 +325,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
@Test
|
||||
public void testBulkInsertPreppedAndCleanByCommits() throws Exception {
|
||||
testInsertAndCleanByCommits(
|
||||
(client, recordRDD, commitTime) -> client.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()),
|
||||
(client, recordRDD, instantTime) -> client.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()),
|
||||
HoodieWriteClient::upsertPreppedRecords, true);
|
||||
}
|
||||
|
||||
@@ -598,7 +598,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
|
||||
@Test
|
||||
public void testUpgradeDowngrade() {
|
||||
String commitTime = "000";
|
||||
String instantTime = "000";
|
||||
|
||||
String partition1 = DEFAULT_PARTITION_PATHS[0];
|
||||
String partition2 = DEFAULT_PARTITION_PATHS[1];
|
||||
@@ -616,7 +616,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
// create partition1 clean stat.
|
||||
HoodieCleanStat cleanStat1 = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS,
|
||||
partition1, deletePathPatterns1, successDeleteFiles1,
|
||||
failedDeleteFiles1, commitTime);
|
||||
failedDeleteFiles1, instantTime);
|
||||
|
||||
List<String> deletePathPatterns2 = new ArrayList<>();
|
||||
List<String> successDeleteFiles2 = new ArrayList<>();
|
||||
@@ -625,7 +625,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
// create partition2 empty clean stat.
|
||||
HoodieCleanStat cleanStat2 = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_COMMITS,
|
||||
partition2, deletePathPatterns2, successDeleteFiles2,
|
||||
failedDeleteFiles2, commitTime);
|
||||
failedDeleteFiles2, instantTime);
|
||||
|
||||
// map with absolute file path.
|
||||
Map<String, Tuple3> oldExpected = new HashMap<>();
|
||||
@@ -639,7 +639,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
newExpected.put(partition2, new Tuple3<>(deletePathPatterns2, successDeleteFiles2, failedDeleteFiles2));
|
||||
|
||||
HoodieCleanMetadata metadata =
|
||||
CleanerUtils.convertCleanMetadata(metaClient, commitTime, Option.of(0L), Arrays.asList(cleanStat1, cleanStat2));
|
||||
CleanerUtils.convertCleanMetadata(metaClient, instantTime, Option.of(0L), Arrays.asList(cleanStat1, cleanStat2));
|
||||
metadata.setVersion(CleanerUtils.CLEAN_METADATA_VERSION_1);
|
||||
|
||||
// NOw upgrade and check
|
||||
@@ -1107,15 +1107,15 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
/**
|
||||
* Utility method to create temporary data files.
|
||||
*
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param instantTime Commit Timestamp
|
||||
* @param numFiles Number for files to be generated
|
||||
* @return generated files
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
private List<String> createMarkerFiles(String commitTime, int numFiles) throws IOException {
|
||||
private List<String> createMarkerFiles(String instantTime, int numFiles) throws IOException {
|
||||
List<String> files = new ArrayList<>();
|
||||
for (int i = 0; i < numFiles; i++) {
|
||||
files.add(HoodieTestUtils.createNewMarkerFile(basePath, "2019/03/29", commitTime));
|
||||
files.add(HoodieTestUtils.createNewMarkerFile(basePath, "2019/03/29", instantTime));
|
||||
}
|
||||
return files;
|
||||
}
|
||||
|
||||
@@ -93,7 +93,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String partitionPath = "2016/05/04";
|
||||
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
String instantTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
@@ -103,12 +103,12 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
when(record.getPartitionPath()).thenReturn(partitionPath);
|
||||
String writeToken = FSUtils.makeWriteToken(TaskContext.getPartitionId(), TaskContext.get().stageId(),
|
||||
TaskContext.get().taskAttemptId());
|
||||
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath, fileName);
|
||||
HoodieCreateHandle io = new HoodieCreateHandle(config, instantTime, table, partitionPath, fileName);
|
||||
return Pair.of(io.makeNewPath(record.getPartitionPath()), writeToken);
|
||||
}).collect().get(0);
|
||||
|
||||
Assert.assertEquals(newPathWithWriteToken.getKey().toString(), this.basePath + "/" + partitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime, newPathWithWriteToken.getRight(), fileName));
|
||||
+ FSUtils.makeDataFileName(instantTime, newPathWithWriteToken.getRight(), fileName));
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeHoodieClientConfig() throws Exception {
|
||||
@@ -306,7 +306,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
@Test
|
||||
public void testInsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
String instantTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
@@ -318,7 +318,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
// Insert new records
|
||||
final List<HoodieRecord> recs2 = records;
|
||||
List<WriteStatus> returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return table.handleInsert(commitTime, FSUtils.createNewFileIdPfx(), recs2.iterator());
|
||||
return table.handleInsert(instantTime, FSUtils.createNewFileIdPfx(), recs2.iterator());
|
||||
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
|
||||
|
||||
// TODO: check the actual files and make sure 11 records, total were written.
|
||||
@@ -340,7 +340,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
final List<HoodieRecord> recs3 = records;
|
||||
|
||||
returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return table.handleInsert(commitTime, FSUtils.createNewFileIdPfx(), recs3.iterator());
|
||||
return table.handleInsert(instantTime, FSUtils.createNewFileIdPfx(), recs3.iterator());
|
||||
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
|
||||
|
||||
assertEquals(3, returnedStatuses.size());
|
||||
@@ -359,7 +359,7 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
public void testFileSizeUpsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.limitFileSize(64 * 1024).parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024).build()).build();
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
String instantTime = HoodieTestUtils.makeNewCommitTime();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
|
||||
@@ -374,13 +374,13 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
|
||||
// Insert new records
|
||||
jsc.parallelize(Arrays.asList(1))
|
||||
.map(i -> table.handleInsert(commitTime, FSUtils.createNewFileIdPfx(), records.iterator()))
|
||||
.map(i -> table.handleInsert(instantTime, FSUtils.createNewFileIdPfx(), records.iterator()))
|
||||
.map(x -> HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
|
||||
// Check the updated file
|
||||
int counts = 0;
|
||||
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
|
||||
if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()).equals(commitTime)) {
|
||||
if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()).equals(instantTime)) {
|
||||
LOG.info(file.getName() + "-" + file.length());
|
||||
counts++;
|
||||
}
|
||||
@@ -471,11 +471,11 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
final HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
String commitTime = "000";
|
||||
String instantTime = "000";
|
||||
// Perform inserts of 100 records to test CreateHandle and BufferedExecutor
|
||||
final List<HoodieRecord> inserts = dataGen.generateInsertsWithHoodieAvroPayload(commitTime, 100);
|
||||
final List<HoodieRecord> inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100);
|
||||
final List<List<WriteStatus>> ws = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return table.handleInsert(commitTime, UUID.randomUUID().toString(), inserts.iterator());
|
||||
return table.handleInsert(instantTime, UUID.randomUUID().toString(), inserts.iterator());
|
||||
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
|
||||
WriteStatus writeStatus = ws.get(0).get(0);
|
||||
@@ -483,12 +483,12 @@ public class TestCopyOnWriteTable extends HoodieClientTestHarness {
|
||||
metaClient.getFs().create(new Path(basePath + "/.hoodie/000.commit")).close();
|
||||
final HoodieCopyOnWriteTable table2 = new HoodieCopyOnWriteTable(config, jsc);
|
||||
final List<HoodieRecord> updates =
|
||||
dataGen.generateUpdatesWithHoodieAvroPayload(commitTime, inserts);
|
||||
dataGen.generateUpdatesWithHoodieAvroPayload(instantTime, inserts);
|
||||
|
||||
String partitionPath = updates.get(0).getPartitionPath();
|
||||
long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count();
|
||||
final List<List<WriteStatus>> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return table.handleUpdate(commitTime, partitionPath, fileId, updates.iterator());
|
||||
return table.handleUpdate(instantTime, partitionPath, fileId, updates.iterator());
|
||||
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
assertEquals(updates.size() - numRecordsInPartition, updateStatus.get(0).get(0).getTotalErrorRecords());
|
||||
}
|
||||
|
||||
@@ -839,11 +839,11 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
|
||||
Assert.assertTrue(numLogFiles > 0);
|
||||
// Do a compaction
|
||||
String commitTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
statuses = writeClient.compact(commitTime);
|
||||
String instantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
statuses = writeClient.compact(instantTime);
|
||||
assertEquals(statuses.map(status -> status.getStat().getPath().contains("parquet")).count(), numLogFiles);
|
||||
Assert.assertEquals(statuses.count(), numLogFiles);
|
||||
writeClient.commitCompaction(commitTime, statuses, Option.empty());
|
||||
writeClient.commitCompaction(instantTime, statuses, Option.empty());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -991,14 +991,14 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
instant = new HoodieInstant(State.INFLIGHT, commitActionType, "000");
|
||||
activeTimeline.saveAsComplete(instant, Option.empty());
|
||||
|
||||
String commitTime = "001";
|
||||
client.startCommitWithTime(commitTime);
|
||||
String instantTime = "001";
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(instantTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> statuses = client.insert(writeRecords, commitTime);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, statuses));
|
||||
JavaRDD<WriteStatus> statuses = client.insert(writeRecords, instantTime);
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
@@ -1018,12 +1018,12 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
Assert.assertEquals(inserts, 200);
|
||||
|
||||
commitTime = "002";
|
||||
client.startCommitWithTime(commitTime);
|
||||
records = dataGen.generateUpdates(commitTime, records);
|
||||
instantTime = "002";
|
||||
client.startCommitWithTime(instantTime);
|
||||
records = dataGen.generateUpdates(instantTime, records);
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
statuses = client.upsert(writeRecords, commitTime);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, statuses));
|
||||
statuses = client.upsert(writeRecords, instantTime);
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
@@ -1047,7 +1047,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
Assert.assertEquals(inserts, 200);
|
||||
Assert.assertEquals(upserts, 200);
|
||||
|
||||
client.rollback(commitTime);
|
||||
client.rollback(instantTime);
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
@@ -1084,14 +1084,14 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
Map<String, Long> fileIdToInsertsMap = new HashMap<>();
|
||||
Map<String, Long> fileIdToUpsertsMap = new HashMap<>();
|
||||
|
||||
String commitTime = "000";
|
||||
client.startCommitWithTime(commitTime);
|
||||
String instantTime = "000";
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(instantTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> statuses = client.insert(writeRecords, commitTime);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, statuses));
|
||||
JavaRDD<WriteStatus> statuses = client.insert(writeRecords, instantTime);
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
@@ -1113,14 +1113,14 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
Assert.assertEquals(inserts, 200);
|
||||
|
||||
commitTime = "001";
|
||||
client.startCommitWithTime(commitTime);
|
||||
instantTime = "001";
|
||||
client.startCommitWithTime(instantTime);
|
||||
// generate updates + inserts. inserts should be handled into small files
|
||||
records = dataGen.generateUpdates(commitTime, records);
|
||||
records.addAll(dataGen.generateInserts(commitTime, 200));
|
||||
records = dataGen.generateUpdates(instantTime, records);
|
||||
records.addAll(dataGen.generateInserts(instantTime, 200));
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
statuses = client.upsert(writeRecords, commitTime);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, statuses));
|
||||
statuses = client.upsert(writeRecords, instantTime);
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
@@ -1148,10 +1148,10 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
Assert.assertEquals(upserts, 200);
|
||||
|
||||
// Test small file handling after compaction
|
||||
commitTime = "002";
|
||||
client.scheduleCompactionAtInstant(commitTime, Option.of(metadata.getExtraMetadata()));
|
||||
statuses = client.compact(commitTime);
|
||||
client.commitCompaction(commitTime, statuses, Option.empty());
|
||||
instantTime = "002";
|
||||
client.scheduleCompactionAtInstant(instantTime, Option.of(metadata.getExtraMetadata()));
|
||||
statuses = client.compact(instantTime);
|
||||
client.commitCompaction(instantTime, statuses, Option.empty());
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
@@ -1172,14 +1172,14 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
|
||||
// Write inserts + updates
|
||||
commitTime = "003";
|
||||
client.startCommitWithTime(commitTime);
|
||||
instantTime = "003";
|
||||
client.startCommitWithTime(instantTime);
|
||||
// generate updates + inserts. inserts should be handled into small files
|
||||
records = dataGen.generateUpdates(commitTime, records);
|
||||
records.addAll(dataGen.generateInserts(commitTime, 200));
|
||||
records = dataGen.generateUpdates(instantTime, records);
|
||||
records.addAll(dataGen.generateInserts(instantTime, 200));
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
statuses = client.upsert(writeRecords, commitTime);
|
||||
assertTrue("Commit should succeed", client.commit(commitTime, statuses));
|
||||
statuses = client.upsert(writeRecords, instantTime);
|
||||
assertTrue("Commit should succeed", client.commit(instantTime, statuses));
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
Reference in New Issue
Block a user