1
0

Adding ability for inserts to be written to log files

This commit is contained in:
Nishith Agarwal
2018-05-13 16:25:11 -07:00
committed by vinoth chandar
parent 34827d50e1
commit 3da063f83b
52 changed files with 1061 additions and 519 deletions

View File

@@ -94,10 +94,11 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
this.jsc = jsc; this.jsc = jsc;
this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
this.hoodieTable = HoodieTable this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), .getHoodieTable(metaClient,
clientConfig); clientConfig, jsc);
this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants(); this.commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
this.index = HoodieIndex.createIndex(clientConfig, jsc); this.index = HoodieIndex.createIndex(clientConfig, jsc);
this.sqlContextOpt = Optional.absent(); this.sqlContextOpt = Optional.absent();
} }
@@ -128,7 +129,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
assertSqlContext(); assertSqlContext();
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD = index JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD = index
.fetchRecordLocation(hoodieKeys, hoodieTable); .fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
List<String> paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent()) List<String> paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent())
.map(keyFileTuple -> keyFileTuple._2().get()).collect(); .map(keyFileTuple -> keyFileTuple._2().get()).collect();
@@ -156,7 +157,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
* file * file
*/ */
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) { public JavaPairRDD<HoodieKey, Optional<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
return index.fetchRecordLocation(hoodieKeys, hoodieTable); return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
} }
/** /**
@@ -180,6 +181,6 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
*/ */
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords) public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords)
throws HoodieIndexException { throws HoodieIndexException {
return index.tagLocation(hoodieRecords, hoodieTable); return index.tagLocation(hoodieRecords, jsc, hoodieTable);
} }
} }

View File

@@ -130,7 +130,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public static SparkConf registerClasses(SparkConf conf) { public static SparkConf registerClasses(SparkConf conf) {
conf.registerKryoClasses( conf.registerKryoClasses(
new Class[] {HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
return conf; return conf;
} }
@@ -144,9 +144,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) { public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, table); JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, jsc, table);
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
} }
@@ -161,7 +161,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
config.shouldCombineBeforeUpsert(), records, config.getUpsertShuffleParallelism()); config.shouldCombineBeforeUpsert(), records, config.getUpsertShuffleParallelism());
// perform index loop up to get existing location of records // perform index loop up to get existing location of records
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, table); JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, jsc, table);
return upsertRecordsInternal(taggedRecords, commitTime, table, true); return upsertRecordsInternal(taggedRecords, commitTime, table, true);
} catch (Throwable e) { } catch (Throwable e) {
if (e instanceof HoodieUpsertException) { if (e instanceof HoodieUpsertException) {
@@ -178,7 +178,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* needed. * needed.
* *
* @param preppedRecords Prepared HoodieRecords to upsert * @param preppedRecords Prepared HoodieRecords to upsert
* @param commitTime Commit Time handle * @param commitTime Commit Time handle
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
*/ */
public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords,
@@ -202,7 +202,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* This implementation skips the index check and is able to leverage benefits such as small file * This implementation skips the index check and is able to leverage benefits such as small file
* handling/blocking alignment, as with upsert(), by profiling the workload * handling/blocking alignment, as with upsert(), by profiling the workload
* *
* @param records HoodieRecords to insert * @param records HoodieRecords to insert
* @param commitTime Commit Time handle * @param commitTime Commit Time handle
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
*/ */
@@ -230,7 +230,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* The prepared HoodieRecords should be de-duped if needed. * The prepared HoodieRecords should be de-duped if needed.
* *
* @param preppedRecords HoodieRecords to insert * @param preppedRecords HoodieRecords to insert
* @param commitTime Commit Time handle * @param commitTime Commit Time handle
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
*/ */
public JavaRDD<WriteStatus> insertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, public JavaRDD<WriteStatus> insertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords,
@@ -256,7 +256,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* attempts to control the numbers of files with less memory compared to the {@link * attempts to control the numbers of files with less memory compared to the {@link
* HoodieWriteClient#insert(JavaRDD, String)} * HoodieWriteClient#insert(JavaRDD, String)}
* *
* @param records HoodieRecords to insert * @param records HoodieRecords to insert
* @param commitTime Commit Time handle * @param commitTime Commit Time handle
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
*/ */
@@ -276,10 +276,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* partitioner. If specified then it will be used for repartitioning records. See {@link * partitioner. If specified then it will be used for repartitioning records. See {@link
* UserDefinedBulkInsertPartitioner}. * UserDefinedBulkInsertPartitioner}.
* *
* @param records HoodieRecords to insert * @param records HoodieRecords to insert
* @param commitTime Commit Time handle * @param commitTime Commit Time handle
* @param bulkInsertPartitioner If specified then it will be used to partition input records * @param bulkInsertPartitioner If specified then it will be used to partition input records
* before they are inserted into hoodie. * before they are inserted into hoodie.
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts * @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 commitTime,
@@ -310,10 +310,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* partitioner. If specified then it will be used for repartitioning records. See {@link * partitioner. If specified then it will be used for repartitioning records. See {@link
* UserDefinedBulkInsertPartitioner}. * UserDefinedBulkInsertPartitioner}.
* *
* @param preppedRecords HoodieRecords to insert * @param preppedRecords HoodieRecords to insert
* @param commitTime Commit Time handle * @param commitTime Commit Time handle
* @param bulkInsertPartitioner If specified then it will be used to partition input records * @param bulkInsertPartitioner If specified then it will be used to partition input records
* before they are inserted into hoodie. * before they are inserted into hoodie.
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
*/ */
public JavaRDD<WriteStatus> bulkInsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, public JavaRDD<WriteStatus> bulkInsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords,
@@ -390,6 +390,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
partitionStat.getUpdateLocationToCount().entrySet().stream().forEach(entry -> { partitionStat.getUpdateLocationToCount().entrySet().stream().forEach(entry -> {
HoodieWriteStat writeStat = new HoodieWriteStat(); HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId(entry.getKey()); writeStat.setFileId(entry.getKey());
// TODO : Write baseCommitTime is possible here ?
writeStat.setPrevCommit(entry.getValue().getKey()); writeStat.setPrevCommit(entry.getValue().getKey());
writeStat.setNumUpdateWrites(entry.getValue().getValue()); writeStat.setNumUpdateWrites(entry.getValue().getValue());
metadata.addWriteStat(path.toString(), writeStat); metadata.addWriteStat(path.toString(), writeStat);
@@ -450,10 +451,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> table, String commitTime) { HoodieTable<T> table, String commitTime) {
// Update the index back // Update the index back
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, table); JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, jsc, table);
// Trigger the insert and collect statuses // Trigger the insert and collect statuses
statuses = statuses.persist(config.getWriteStatusStorageLevel()); statuses = statuses.persist(config.getWriteStatusStorageLevel());
commitOnAutoCommit(commitTime, statuses, table.getCommitActionType()); commitOnAutoCommit(commitTime, statuses,
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)
.getCommitActionType());
return statuses; return statuses;
} }
@@ -476,9 +479,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/ */
public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses, public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses,
Optional<HashMap<String, String>> extraMetadata) { Optional<HashMap<String, String>> extraMetadata) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true);
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); return commit(commitTime, writeStatuses, extraMetadata, metaClient.getCommitActionType());
return commit(commitTime, writeStatuses, extraMetadata, table.getCommitActionType());
} }
private boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses, private boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses,
@@ -487,7 +489,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
logger.info("Commiting " + commitTime); logger.info("Commiting " + commitTime);
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
@@ -531,7 +533,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// We cannot have unbounded commit files. Archive commits if we have to archive // We cannot have unbounded commit files. Archive commits if we have to archive
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config, HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config,
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true));
archiveLog.archiveIfRequired(); archiveLog.archiveIfRequired(jsc);
if (config.isAutoClean()) { if (config.isAutoClean()) {
// Call clean to cleanup if there is anything to cleanup after the commit, // Call clean to cleanup if there is anything to cleanup after the commit,
logger.info("Auto cleaning is enabled. Running cleaner now"); logger.info("Auto cleaning is enabled. Running cleaner now");
@@ -568,13 +570,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* <p> * <p>
* Savepoint should be on a commit that could not have been cleaned. * Savepoint should be on a commit that could not have been cleaned.
* *
* @param user - User creating the savepoint * @param user - User creating the savepoint
* @param comment - Comment for the savepoint * @param comment - Comment for the savepoint
* @return true if the savepoint was created successfully * @return true if the savepoint was created successfully
*/ */
public boolean savepoint(String user, String comment) { public boolean savepoint(String user, String comment) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
if (table.getCompletedCommitTimeline().empty()) { if (table.getCompletedCommitTimeline().empty()) {
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
} }
@@ -595,13 +597,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* Savepoint should be on a commit that could not have been cleaned. * Savepoint should be on a commit that could not have been cleaned.
* *
* @param commitTime - commit that should be savepointed * @param commitTime - commit that should be savepointed
* @param user - User creating the savepoint * @param user - User creating the savepoint
* @param comment - Comment for the savepoint * @param comment - Comment for the savepoint
* @return true if the savepoint was created successfully * @return true if the savepoint was created successfully
*/ */
public boolean savepoint(String commitTime, String user, String comment) { public boolean savepoint(String commitTime, String user, String comment) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant(); Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
@@ -662,7 +664,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/ */
public void deleteSavepoint(String savepointTime) { public void deleteSavepoint(String savepointTime) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION,
@@ -688,9 +690,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/ */
public boolean rollbackToSavepoint(String savepointTime) { public boolean rollbackToSavepoint(String savepointTime) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieTimeline commitTimeline = table.getCommitsTimeline(); HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline();
HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION,
savepointTime); savepointTime);
@@ -737,7 +739,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieTimeline inflightTimeline = table.getInflightCommitTimeline(); HoodieTimeline inflightTimeline = table.getInflightCommitTimeline();
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
@@ -841,7 +843,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
List<HoodieCleanStat> cleanStats = table.clean(jsc); List<HoodieCleanStat> cleanStats = table.clean(jsc);
if (cleanStats.isEmpty()) { if (cleanStats.isEmpty()) {
@@ -890,9 +892,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public void startCommitWithTime(String commitTime) { public void startCommitWithTime(String commitTime) {
logger.info("Generate a new commit time " + commitTime); logger.info("Generate a new commit time " + commitTime);
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
String commitActionType = table.getCommitActionType(); String commitActionType = table.getMetaClient().getCommitActionType();
activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime));
} }
@@ -912,7 +914,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/ */
public void startCompactionWithTime(String commitTime) { public void startCompactionWithTime(String commitTime) {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
String commitActionType = HoodieTimeline.COMMIT_ACTION; String commitActionType = HoodieTimeline.COMMIT_ACTION;
activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime));
@@ -925,7 +927,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public JavaRDD<WriteStatus> compact(String commitTime) throws IOException { public JavaRDD<WriteStatus> compact(String commitTime) throws IOException {
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
// TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction // TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction
writeContext = metrics.getCommitCtx(); writeContext = metrics.getCommitCtx();
JavaRDD<WriteStatus> statuses = table.compact(jsc, commitTime); JavaRDD<WriteStatus> statuses = table.compact(jsc, commitTime);
@@ -961,7 +963,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
config.getBasePath(), true); config.getBasePath(), true);
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);
// TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction and // TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction and
// then use getTableAndInitCtx // then use getTableAndInitCtx
Timer.Context writeContext = metrics.getCommitCtx(); Timer.Context writeContext = metrics.getCommitCtx();
@@ -1048,8 +1050,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/ */
private void rollbackInflightCommits() { private void rollbackInflightCommits() {
HoodieTable<T> table = HoodieTable.getHoodieTable( HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterInflights();
List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList()); .collect(Collectors.toList());
Collections.reverse(commits); Collections.reverse(commits);
@@ -1061,8 +1063,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
private HoodieTable getTableAndInitCtx() { private HoodieTable getTableAndInitCtx() {
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable table = HoodieTable.getHoodieTable( HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
if (table.getCommitActionType() == HoodieTimeline.COMMIT_ACTION) { if (table.getMetaClient().getCommitActionType() == HoodieTimeline.COMMIT_ACTION) {
writeContext = metrics.getCommitCtx(); writeContext = metrics.getCommitCtx();
} else { } else {
writeContext = metrics.getDeltaCommitCtx(); writeContext = metrics.getDeltaCommitCtx();

View File

@@ -53,10 +53,10 @@ public class WriteStatus implements Serializable {
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus * aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
* objects are collected in Spark Driver. * objects are collected in Spark Driver.
* *
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies * @param record deflated {@code HoodieRecord} containing information that uniquely identifies
* it. * it.
* @param optionalRecordMetadata optional metadata related to data contained in {@link * @param optionalRecordMetadata optional metadata related to data contained in {@link
* HoodieRecord} before deflation. * HoodieRecord} before deflation.
*/ */
public void markSuccess(HoodieRecord record, public void markSuccess(HoodieRecord record,
Optional<Map<String, String>> optionalRecordMetadata) { Optional<Map<String, String>> optionalRecordMetadata) {
@@ -69,10 +69,10 @@ public class WriteStatus implements Serializable {
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus * aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
* objects are collected in Spark Driver. * objects are collected in Spark Driver.
* *
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies * @param record deflated {@code HoodieRecord} containing information that uniquely identifies
* it. * it.
* @param optionalRecordMetadata optional metadata related to data contained in {@link * @param optionalRecordMetadata optional metadata related to data contained in {@link
* HoodieRecord} before deflation. * HoodieRecord} before deflation.
*/ */
public void markFailure(HoodieRecord record, Throwable t, public void markFailure(HoodieRecord record, Throwable t,
Optional<Map<String, String>> optionalRecordMetadata) { Optional<Map<String, String>> optionalRecordMetadata) {

View File

@@ -44,6 +44,9 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio"; public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio";
// Default compression ratio for parquet // Default compression ratio for parquet
public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1); public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1);
public static final String LOGFILE_TO_PARQUET_COMPRESSION_RATIO = "hoodie.logfile.to.parquet.compression.ratio";
// Default compression ratio for log file to parquet, general 3x
public static final String DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO = String.valueOf(0.35);
private HoodieStorageConfig(Properties props) { private HoodieStorageConfig(Properties props) {
super(props); super(props);
@@ -102,6 +105,11 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
return this; return this;
} }
public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressionRatio) {
props.setProperty(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio));
return this;
}
public HoodieStorageConfig build() { public HoodieStorageConfig build() {
HoodieStorageConfig config = new HoodieStorageConfig(props); HoodieStorageConfig config = new HoodieStorageConfig(props);
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES), setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES),
@@ -116,6 +124,8 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
LOGFILE_SIZE_MAX_BYTES, DEFAULT_LOGFILE_SIZE_MAX_BYTES); LOGFILE_SIZE_MAX_BYTES, DEFAULT_LOGFILE_SIZE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO), setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO),
PARQUET_COMPRESSION_RATIO, DEFAULT_STREAM_COMPRESSION_RATIO); PARQUET_COMPRESSION_RATIO, DEFAULT_STREAM_COMPRESSION_RATIO);
setDefaultOnCondition(props, !props.containsKey(LOGFILE_TO_PARQUET_COMPRESSION_RATIO),
LOGFILE_TO_PARQUET_COMPRESSION_RATIO, DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO);
return config; return config;
} }
} }

View File

@@ -322,6 +322,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return Double.valueOf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO)); return Double.valueOf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO));
} }
public double getLogFileToParquetCompressionRatio() {
return Double.valueOf(props.getProperty(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO));
}
/** /**
* metrics properties * metrics properties
**/ **/
@@ -345,7 +349,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
public String getGraphiteMetricPrefix() { public String getGraphiteMetricPrefix() {
return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX); return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
} }
/** /**
* memory configs * memory configs
*/ */

View File

@@ -46,6 +46,6 @@ public class BulkInsertMapFunction<T extends HoodieRecordPayload> implements
@Override @Override
public Iterator<List<WriteStatus>> call(Integer partition, public Iterator<List<WriteStatus>> call(Integer partition,
Iterator<HoodieRecord<T>> sortedRecordItr) throws Exception { Iterator<HoodieRecord<T>> sortedRecordItr) throws Exception {
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable); return new CopyOnWriteLazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable);
} }
} }

View File

@@ -33,6 +33,7 @@ import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Optional; import java.util.Optional;
import java.util.Set; import java.util.Set;
import java.util.UUID;
import java.util.function.Function; import java.util.function.Function;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
@@ -43,15 +44,15 @@ import scala.Tuple2;
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
* files. * files.
*/ */
public class LazyInsertIterable<T extends HoodieRecordPayload> extends public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extends
LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> { LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
private final HoodieWriteConfig hoodieConfig; protected final HoodieWriteConfig hoodieConfig;
private final String commitTime; protected final String commitTime;
private final HoodieTable<T> hoodieTable; protected final HoodieTable<T> hoodieTable;
private Set<String> partitionsCleaned; protected Set<String> partitionsCleaned;
public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config, public CopyOnWriteLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
String commitTime, HoodieTable<T> hoodieTable) { String commitTime, HoodieTable<T> hoodieTable) {
super(sortedRecordItr); super(sortedRecordItr);
this.partitionsCleaned = new HashSet<>(); this.partitionsCleaned = new HashSet<>();
@@ -89,7 +90,7 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig); final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig);
bufferedIteratorExecutor = bufferedIteratorExecutor =
new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr,
new InsertHandler(), getTransformFunction(schema)); getInsertHandler(), getTransformFunction(schema));
final List<WriteStatus> result = bufferedIteratorExecutor.execute(); final List<WriteStatus> result = bufferedIteratorExecutor.execute();
assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
return result; return result;
@@ -107,15 +108,19 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
} }
protected CopyOnWriteInsertHandler getInsertHandler() {
return new CopyOnWriteInsertHandler();
}
/** /**
* Consumes stream of hoodie records from in-memory queue and * Consumes stream of hoodie records from in-memory queue and
* writes to one or more create-handles * writes to one or more create-handles
*/ */
private class InsertHandler extends protected class CopyOnWriteInsertHandler extends
BoundedInMemoryQueueConsumer<Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>, List<WriteStatus>> { BoundedInMemoryQueueConsumer<Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>, List<WriteStatus>> {
private final List<WriteStatus> statuses = new ArrayList<>(); protected final List<WriteStatus> statuses = new ArrayList<>();
private HoodieCreateHandle handle; protected HoodieIOHandle handle;
@Override @Override
protected void consumeOneRecord(Tuple2<HoodieRecord<T>, Optional<IndexedRecord>> payload) { protected void consumeOneRecord(Tuple2<HoodieRecord<T>, Optional<IndexedRecord>> payload) {
@@ -132,7 +137,8 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
// lazily initialize the handle, for the first time // lazily initialize the handle, for the first time
if (handle == null) { if (handle == null) {
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), UUID
.randomUUID().toString());
} }
if (handle.canWrite(payload._1())) { if (handle.canWrite(payload._1())) {
@@ -142,7 +148,8 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
// handle is full. // handle is full.
statuses.add(handle.close()); statuses.add(handle.close());
// Need to handle the rejected payload & open new handle // Need to handle the rejected payload & open new handle
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), UUID
.randomUUID().toString());
handle.write(insertPayload, payload._2()); // we should be able to write 1 payload. handle.write(insertPayload, payload._2()); // we should be able to write 1 payload.
} }
} }
@@ -161,4 +168,4 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
return statuses; return statuses;
} }
} }
} }

View File

@@ -0,0 +1,73 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.func;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import org.apache.avro.generic.IndexedRecord;
import scala.Tuple2;
/**
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
* log files.
*/
public class MergeOnReadLazyInsertIterable<T extends HoodieRecordPayload> extends
CopyOnWriteLazyInsertIterable<T> {
public MergeOnReadLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
String commitTime, HoodieTable<T> hoodieTable) {
super(sortedRecordItr, config, commitTime, hoodieTable);
}
@Override
protected CopyOnWriteInsertHandler getInsertHandler() {
return new MergeOnReadInsertHandler();
}
protected class MergeOnReadInsertHandler extends CopyOnWriteInsertHandler {
@Override
protected void consumeOneRecord(Tuple2<HoodieRecord<T>, Optional<IndexedRecord>> payload) {
final HoodieRecord insertPayload = payload._1();
List<WriteStatus> statuses = new ArrayList<>();
// lazily initialize the handle, for the first time
if (handle == null) {
handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable);
}
if (handle.canWrite(insertPayload)) {
// write the payload, if the handle has capacity
handle.write(insertPayload, payload._2);
} else {
// handle is full.
handle.close();
statuses.add(handle.getWriteStatus());
// Need to handle the rejected payload & open new handle
handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable);
handle.write(insertPayload, payload._2); // we should be able to write 1 payload.
}
}
}
}

View File

@@ -38,24 +38,23 @@ import org.apache.spark.api.java.JavaSparkContext;
public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable { public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
protected final HoodieWriteConfig config; protected final HoodieWriteConfig config;
protected transient JavaSparkContext jsc = null;
protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) { protected HoodieIndex(HoodieWriteConfig config) {
this.config = config; this.config = config;
this.jsc = jsc;
} }
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(HoodieWriteConfig config, public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(HoodieWriteConfig config,
JavaSparkContext jsc) throws HoodieIndexException { JavaSparkContext jsc) throws HoodieIndexException {
switch (config.getIndexType()) { switch (config.getIndexType()) {
case HBASE: case HBASE:
return new HBaseIndex<>(config, jsc); return new HBaseIndex<>(config);
case INMEMORY: case INMEMORY:
return new InMemoryHashIndex<>(config, jsc); return new InMemoryHashIndex<>(config);
case BLOOM: case BLOOM:
return new HoodieBloomIndex<>(config, jsc); return new HoodieBloomIndex<>(config);
case BUCKETED: case BUCKETED:
return new BucketedIndex<>(config, jsc); return new BucketedIndex<>(config);
default: default:
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
} }
@@ -68,22 +67,23 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
* file * file
*/ */
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation( public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table); JavaRDD<HoodieKey> hoodieKeys, final JavaSparkContext jsc, HoodieTable<T> hoodieTable);
/** /**
* Looks up the index and tags each incoming record with a location of a file that contains the * Looks up the index and tags each incoming record with a location of a file that contains the
* row (if it is actually present) * row (if it is actually present)
*/ */
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException; JavaSparkContext jsc, HoodieTable<T> hoodieTable) throws HoodieIndexException;
/** /**
* Extracts the location of written records, and updates the index. * Extracts the location of written records, and updates the index.
* <p> * <p>
* TODO(vc): We may need to propagate the record as well in a WriteStatus class * TODO(vc): We may need to propagate the record as well in a WriteStatus class
*/ */
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) throws HoodieIndexException; HoodieTable<T> hoodieTable)
throws HoodieIndexException;
/** /**
* Rollback the efffects of the commit made at commitTime. * Rollback the efffects of the commit made at commitTime.

View File

@@ -43,25 +43,25 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap; private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) { public InMemoryHashIndex(HoodieWriteConfig config) {
super(config, jsc); super(config);
recordLocationMap = new ConcurrentHashMap<>(); recordLocationMap = new ConcurrentHashMap<>();
} }
@Override @Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
final HoodieTable<T> table) { JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
throw new UnsupportedOperationException("InMemory index does not implement check exist yet"); throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
} }
@Override @Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) { HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true); return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
} }
@Override @Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) { HoodieTable<T> hoodieTable) {
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() { return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
@Override @Override

View File

@@ -30,6 +30,7 @@ import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils; import com.uber.hoodie.common.util.ParquetUtils;
@@ -64,13 +65,13 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION = private static int MAX_ITEMS_PER_SHUFFLE_PARTITION =
SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET; SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) { public HoodieBloomIndex(HoodieWriteConfig config) {
super(config, jsc); super(config);
} }
@Override @Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
final HoodieTable<T> hoodieTable) { HoodieTable<T> hoodieTable) {
// Step 0: cache the input record RDD // Step 0: cache the input record RDD
if (config.getBloomIndexUseCaching()) { if (config.getBloomIndexUseCaching()) {
@@ -82,8 +83,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())); .mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
// Lookup indexes for all the partition/recordkey pair // Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable);
hoodieTable);
// Cache the result, for subsequent stages. // Cache the result, for subsequent stages.
if (config.getBloomIndexUseCaching()) { if (config.getBloomIndexUseCaching()) {
@@ -108,13 +108,12 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
} }
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
final HoodieTable<T> table) { JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD = hoodieKeys JavaPairRDD<String, String> partitionRecordKeyPairRDD = hoodieKeys
.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey())); .mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
// Lookup indexes for all the partition/recordkey pair // Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable);
table);
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD = hoodieKeys JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD = hoodieKeys
.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key)); .mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
@@ -125,7 +124,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
String fileName = keyPathTuple._2._2.get(); String fileName = keyPathTuple._2._2.get();
String partitionPath = keyPathTuple._2._1.getPartitionPath(); String partitionPath = keyPathTuple._2._1.getPartitionPath();
recordLocationPath = Optional recordLocationPath = Optional
.of(new Path(new Path(table.getMetaClient().getBasePath(), partitionPath), fileName) .of(new Path(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath), fileName)
.toUri().getPath()); .toUri().getPath());
} else { } else {
recordLocationPath = Optional.absent(); recordLocationPath = Optional.absent();
@@ -139,14 +138,15 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* keys already present and drop the record keys if not present * keys already present and drop the record keys if not present
*/ */
private JavaPairRDD<String, String> lookupIndex( private JavaPairRDD<String, String> lookupIndex(
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTable<T> hoodieTable) { JavaPairRDD<String, String> partitionRecordKeyPairRDD, final JavaSparkContext
jsc, final HoodieTable hoodieTable) {
// Obtain records per partition, in the incoming records // Obtain records per partition, in the incoming records
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey(); Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet()); List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
// Step 2: Load all involved files as <Partition, filename> pairs // Step 2: Load all involved files as <Partition, filename> pairs
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles( List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, jsc,
affectedPartitionPathList, hoodieTable); hoodieTable);
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream() final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList()))); .collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
@@ -154,8 +154,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// that contains it. // that contains it.
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo,
partitionRecordKeyPairRDD); partitionRecordKeyPairRDD);
return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo, return findMatchingFilesForRecordKeys(partitionToFileInfo,
partitionRecordKeyPairRDD, parallelism); partitionRecordKeyPairRDD, parallelism, hoodieTable.getMetaClient());
} }
/** /**
@@ -232,12 +232,12 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Load all involved files as <Partition, filename> pair RDD. * Load all involved files as <Partition, filename> pair RDD.
*/ */
@VisibleForTesting @VisibleForTesting
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
final HoodieTable<T> hoodieTable) { final HoodieTable hoodieTable) {
// Obtain the latest data files from all the partitions. // Obtain the latest data files from all the partitions.
List<Tuple2<String, HoodieDataFile>> dataFilesList = jsc List<Tuple2<String, HoodieDataFile>> dataFilesList = jsc
.parallelize(partitions, Math.max(partitions.size(), 1)).flatMapToPair(partitionPath -> { .parallelize(partitions, Math.max(partitions.size(), 1)).flatMapToPair(partitionPath -> {
java.util.Optional<HoodieInstant> latestCommitTime = hoodieTable.getCommitsTimeline() java.util.Optional<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
.filterCompletedInstants().lastInstant(); .filterCompletedInstants().lastInstant();
List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>(); List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>();
if (latestCommitTime.isPresent()) { if (latestCommitTime.isPresent()) {
@@ -352,9 +352,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Make sure the parallelism is atleast the groupby parallelism for tagging location * Make sure the parallelism is atleast the groupby parallelism for tagging location
*/ */
@VisibleForTesting @VisibleForTesting
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(HoodieTable hoodieTable, JavaPairRDD<String, String> findMatchingFilesForRecordKeys(
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo, final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int totalSubpartitions) { JavaPairRDD<String, String> partitionRecordKeyPairRDD, int totalSubpartitions, HoodieTableMetaClient metaClient) {
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(),
totalSubpartitions); totalSubpartitions);
@@ -367,7 +367,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.sortByKey(true, joinParallelism); .sortByKey(true, joinParallelism);
return fileSortedTripletRDD.mapPartitionsWithIndex( return fileSortedTripletRDD.mapPartitionsWithIndex(
new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true) new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true)
.flatMap(indexLookupResults -> indexLookupResults.iterator()) .flatMap(indexLookupResults -> indexLookupResults.iterator())
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0) .filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
.flatMapToPair(lookupResult -> { .flatMapToPair(lookupResult -> {
@@ -410,7 +410,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
} }
@Override @Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) { HoodieTable<T> hoodieTable) {
return writeStatusRDD; return writeStatusRDD;
} }

View File

@@ -20,11 +20,11 @@ package com.uber.hoodie.index.bloom;
import com.uber.hoodie.common.BloomFilter; import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.ParquetUtils; import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.func.LazyIterableIterator; import com.uber.hoodie.func.LazyIterableIterator;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
@@ -49,10 +49,10 @@ public class HoodieBloomIndexCheckFunction implements
private final String basePath; private final String basePath;
private final HoodieTable table; private final HoodieTableMetaClient metaClient;
public HoodieBloomIndexCheckFunction(HoodieTable table, String basePath) { public HoodieBloomIndexCheckFunction(HoodieTableMetaClient metaClient, String basePath) {
this.table = table; this.metaClient = metaClient;
this.basePath = basePath; this.basePath = basePath;
} }
@@ -115,7 +115,7 @@ public class HoodieBloomIndexCheckFunction implements
try { try {
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName); Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
bloomFilter = ParquetUtils bloomFilter = ParquetUtils
.readBloomFilterFromParquetMetadata(table.getHadoopConf(), filePath); .readBloomFilterFromParquetMetadata(metaClient.getHadoopConf(), filePath);
candidateRecordKeys = new ArrayList<>(); candidateRecordKeys = new ArrayList<>();
currentFile = fileName; currentFile = fileName;
currentParitionPath = partitionPath; currentParitionPath = partitionPath;
@@ -163,7 +163,7 @@ public class HoodieBloomIndexCheckFunction implements
.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); .debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
} }
ret.add(new IndexLookupResult(currentFile, ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath))); checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath)));
initState(fileName, partitionPath); initState(fileName, partitionPath);
if (bloomFilter.mightContain(recordKey)) { if (bloomFilter.mightContain(recordKey)) {
@@ -176,7 +176,7 @@ public class HoodieBloomIndexCheckFunction implements
} }
} }
// handle case, where we ran out of input, finish pending work, update return val // handle case, where we ran out of input, close pending work, update return val
if (!inputItr.hasNext()) { if (!inputItr.hasNext()) {
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile); Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
logger.info( logger.info(
@@ -186,7 +186,7 @@ public class HoodieBloomIndexCheckFunction implements
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
} }
ret.add(new IndexLookupResult(currentFile, ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath))); checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath)));
} }
} catch (Throwable e) { } catch (Throwable e) {

View File

@@ -49,8 +49,8 @@ public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T>
private static Logger logger = LogManager.getLogger(BucketedIndex.class); private static Logger logger = LogManager.getLogger(BucketedIndex.class);
public BucketedIndex(HoodieWriteConfig config, JavaSparkContext jsc) { public BucketedIndex(HoodieWriteConfig config) {
super(config, jsc); super(config);
} }
private String getBucket(String recordKey) { private String getBucket(String recordKey) {
@@ -59,13 +59,14 @@ public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T>
@Override @Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
HoodieTable<T> table) { JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey())))); return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey()))));
} }
@Override @Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) throws HoodieIndexException { HoodieTable<T> hoodieTable)
throws HoodieIndexException {
return recordRDD.map(record -> { return recordRDD.map(record -> {
String bucket = getBucket(record.getRecordKey()); String bucket = getBucket(record.getRecordKey());
//HACK(vc) a non-existent commit is provided here. //HACK(vc) a non-existent commit is provided here.
@@ -75,8 +76,9 @@ public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T>
} }
@Override @Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) throws HoodieIndexException { HoodieTable<T> hoodieTable)
throws HoodieIndexException {
return writeStatusRDD; return writeStatusRDD;
} }

View File

@@ -25,6 +25,7 @@ import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
@@ -69,15 +70,15 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
private static Connection hbaseConnection = null; private static Connection hbaseConnection = null;
private final String tableName; private final String tableName;
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) { public HBaseIndex(HoodieWriteConfig config) {
super(config, jsc); super(config);
this.tableName = config.getHbaseTableName(); this.tableName = config.getHbaseTableName();
addShutDownHook(); addShutDownHook();
} }
@Override @Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
HoodieTable<T> table) { JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
//TODO : Change/Remove filterExists in HoodieReadClient() and revisit //TODO : Change/Remove filterExists in HoodieReadClient() and revisit
throw new UnsupportedOperationException("HBase index does not implement check exist"); throw new UnsupportedOperationException("HBase index does not implement check exist");
} }
@@ -119,8 +120,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN); .addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN);
} }
private boolean checkIfValidCommit(HoodieTable<T> hoodieTable, String commitTs) { private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) {
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline(); HoodieTimeline commitTimeline = metaClient.getActiveTimeline().filterCompletedInstants();
// Check if the last commit ts for this row is 1) present in the timeline or // Check if the last commit ts for this row is 1) present in the timeline or
// 2) is less than the first commit ts in the timeline // 2) is less than the first commit ts in the timeline
return !commitTimeline.empty() && (commitTimeline return !commitTimeline.empty() && (commitTimeline
@@ -133,8 +134,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
/** /**
* Function that tags each HoodieRecord with an existing location, if known. * Function that tags each HoodieRecord with an existing location, if known.
*/ */
private Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> private Function2<Integer, Iterator<HoodieRecord<T>>,
locationTagFunction(HoodieTable<T> hoodieTable) { Iterator<HoodieRecord<T>>> locationTagFunction(HoodieTableMetaClient metaClient) {
return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>) return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>)
(partitionNum, hoodieRecordIterator) -> { (partitionNum, hoodieRecordIterator) -> {
@@ -176,7 +177,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
String partitionPath = Bytes String partitionPath = Bytes
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN)); .toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
if (checkIfValidCommit(hoodieTable, commitTs)) { if (checkIfValidCommit(metaClient, commitTs)) {
currentRecord = new HoodieRecord( currentRecord = new HoodieRecord(
new HoodieKey(currentRecord.getRecordKey(), partitionPath), new HoodieKey(currentRecord.getRecordKey(), partitionPath),
currentRecord.getData()); currentRecord.getData());
@@ -211,13 +212,12 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
} }
@Override @Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) { HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable), true); return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true);
} }
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
updateLocationFunction() {
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition, return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition,
statusIterator) -> { statusIterator) -> {
Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize(); Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize();
@@ -306,7 +306,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
} }
@Override @Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) { HoodieTable<T> hoodieTable) {
return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true); return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true);
} }

View File

@@ -31,7 +31,6 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
@@ -45,6 +44,7 @@ import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
@@ -60,21 +60,35 @@ import org.apache.spark.util.SizeEstimator;
public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> { public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class); private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class);
// This acts as the sequenceID for records written
private static AtomicLong recordIndex = new AtomicLong(1); private static AtomicLong recordIndex = new AtomicLong(1);
private final WriteStatus writeStatus; private final WriteStatus writeStatus;
private final String fileId; private final String fileId;
// Buffer for holding records in memory before they are flushed to disk
List<IndexedRecord> recordList = new ArrayList<>(); List<IndexedRecord> recordList = new ArrayList<>();
// Buffer for holding records (to be deleted) in memory before they are flushed to disk
List<String> keysToDelete = new ArrayList<>(); List<String> keysToDelete = new ArrayList<>();
private TableFileSystemView.RealtimeView fileSystemView; private TableFileSystemView.RealtimeView fileSystemView;
private String partitionPath; private String partitionPath;
private Iterator<HoodieRecord<T>> recordItr; private Iterator<HoodieRecord<T>> recordItr;
// Total number of records written during an append
private long recordsWritten = 0; private long recordsWritten = 0;
// Total number of records deleted during an append
private long recordsDeleted = 0; private long recordsDeleted = 0;
// Average record size for a HoodieRecord. This size is updated at the end of every log block flushed to disk
private long averageRecordSize = 0; private long averageRecordSize = 0;
private HoodieLogFile currentLogFile; private HoodieLogFile currentLogFile;
private Writer writer; private Writer writer;
// Flag used to initialize some metadata
private boolean doInit = true; private boolean doInit = true;
// Total number of bytes written during this append phase (an estimation)
private long estimatedNumberOfBytesWritten; private long estimatedNumberOfBytesWritten;
// Number of records that must be written to meet the max block size for a log block
private int numberOfRecords = 0;
// Max block size to limit to for a log block
private int maxBlockSize = config.getLogFileDataBlockMaxSize();
// Header metadata for a log block
private Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable, public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
String fileId, Iterator<HoodieRecord<T>> recordItr) { String fileId, Iterator<HoodieRecord<T>> recordItr) {
@@ -87,43 +101,46 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
this.recordItr = recordItr; this.recordItr = recordItr;
} }
private void init(String partitionPath) { public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable) {
this(config, commitTime, hoodieTable, UUID.randomUUID().toString(), null);
}
// extract some information from the first record private void init(HoodieRecord record) {
FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath) if (doInit) {
.filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)).findFirst() this.partitionPath = record.getPartitionPath();
.get(); // extract some information from the first record
// HACK(vc) This also assumes a base file. It will break, if appending without one. Optional<FileSlice> fileSlice = fileSystemView.getLatestFileSlices(partitionPath)
String latestValidFilePath = fileSlice.getDataFile().get().getFileName(); .filter(fileSlice1 -> fileSlice1.getFileId().equals(fileId)).findFirst();
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath); String baseInstantTime = commitTime;
writeStatus.getStat().setPrevCommit(baseCommitTime); if (fileSlice.isPresent()) {
writeStatus.setFileId(fileId); baseInstantTime = fileSlice.get().getBaseCommitTime();
writeStatus.setPartitionPath(partitionPath); } else {
writeStatus.getStat().setFileId(fileId); // This means there is no base data file, start appending to a new log file
this.partitionPath = partitionPath; fileSlice = Optional.of(new FileSlice(baseInstantTime, this.fileId));
logger.info("New InsertHandle for partition :" + partitionPath);
try { }
this.writer = HoodieLogFormat.newWriterBuilder() writeStatus.getStat().setPrevCommit(baseInstantTime);
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) writeStatus.setFileId(fileId);
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion( writeStatus.setPartitionPath(partitionPath);
fileSlice.getLogFiles().map(logFile -> logFile.getLogVersion()) writeStatus.getStat().setFileId(fileId);
.max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) averageRecordSize = SizeEstimator.estimate(record);
.withSizeThreshold(config.getLogFileMaxSize()).withFs(fs) try {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); this.writer = createLogWriter(fileSlice, baseInstantTime);
this.currentLogFile = writer.getLogFile(); this.currentLogFile = writer.getLogFile();
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion()); ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion());
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize()); ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize());
} catch (Exception e) { } catch (Exception e) {
logger.error("Error in update task at commit " + commitTime, e); logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e); writeStatus.setGlobalError(e);
throw new HoodieUpsertException( throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit " "Failed to initialize HoodieAppendHandle for FileId: " + fileId + " on commit "
+ commitTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() + commitTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath()
+ partitionPath, e); + partitionPath, e);
}
Path path = new Path(partitionPath, writer.getLogFile().getFileName());
writeStatus.getStat().setPath(path.toString());
doInit = false;
} }
Path path = new Path(partitionPath,
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
writeStatus.getStat().setPath(path.toString());
} }
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) { private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
@@ -160,38 +177,11 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
// TODO (NA) - Perform a schema check of current input record with the last schema on log file // TODO (NA) - Perform a schema check of current input record with the last schema on log file
// to make sure we don't append records with older (shorter) schema than already appended // to make sure we don't append records with older (shorter) schema than already appended
public void doAppend() { public void doAppend() {
int maxBlockSize = config.getLogFileDataBlockMaxSize();
int numberOfRecords = 0;
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
while (recordItr.hasNext()) { while (recordItr.hasNext()) {
HoodieRecord record = recordItr.next(); HoodieRecord record = recordItr.next();
// update the new location of the record, so we know where to find it next init(record);
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); flushToDiskIfRequired(record);
if (doInit) { writeToBuffer(record);
init(record.getPartitionPath());
averageRecordSize = SizeEstimator.estimate(record);
doInit = false;
}
// Append if max number of records reached to achieve block size
if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) {
// Recompute averageRecordSize before writing a new block and update existing value with
// avg of new and old
logger.info("AvgRecordSize => " + averageRecordSize);
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2;
doAppend(header);
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
numberOfRecords = 0;
}
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
if (indexedRecord.isPresent()) {
recordList.add(indexedRecord.get());
} else {
keysToDelete.add(record.getRecordKey());
}
numberOfRecords++;
} }
doAppend(header); doAppend(header);
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
@@ -199,6 +189,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
private void doAppend(Map<HoodieLogBlock.HeaderMetadataType, String> header) { private void doAppend(Map<HoodieLogBlock.HeaderMetadataType, String> header) {
try { try {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
if (recordList.size() > 0) { if (recordList.size() > 0) {
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, header)); writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, header));
recordList.clear(); recordList.clear();
@@ -214,11 +206,37 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
} }
} }
public void close() { @Override
public boolean canWrite(HoodieRecord record) {
return config.getParquetMaxFileSize() >= estimatedNumberOfBytesWritten * config
.getLogFileToParquetCompressionRatio();
}
@Override
public void write(HoodieRecord record, Optional<IndexedRecord> insertValue) {
Optional recordMetadata = record.getData().getMetadata();
try { try {
init(record);
flushToDiskIfRequired(record);
writeToBuffer(record);
} catch (Throwable t) {
// Not throwing exception from here, since we don't want to fail the entire job
// for a single record
writeStatus.markFailure(record, t, recordMetadata);
logger.error("Error writing record " + record, t);
}
}
@Override
public WriteStatus close() {
try {
// flush any remaining records to disk
doAppend(header);
if (writer != null) { if (writer != null) {
writer.close(); writer.close();
} }
writeStatus.getStat().setPrevCommit(commitTime);
writeStatus.getStat().setFileId(this.fileId);
writeStatus.getStat().setNumWrites(recordsWritten); writeStatus.getStat().setNumWrites(recordsWritten);
writeStatus.getStat().setNumDeletes(recordsDeleted); writeStatus.getStat().setNumDeletes(recordsDeleted);
writeStatus.getStat().setTotalWriteBytes(estimatedNumberOfBytesWritten); writeStatus.getStat().setTotalWriteBytes(estimatedNumberOfBytesWritten);
@@ -226,13 +244,54 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
RuntimeStats runtimeStats = new RuntimeStats(); RuntimeStats runtimeStats = new RuntimeStats();
runtimeStats.setTotalUpsertTime(timer.endTimer()); runtimeStats.setTotalUpsertTime(timer.endTimer());
writeStatus.getStat().setRuntimeStats(runtimeStats); writeStatus.getStat().setRuntimeStats(runtimeStats);
return writeStatus;
} catch (IOException e) { } catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e); throw new HoodieUpsertException("Failed to close UpdateHandle", e);
} }
} }
@Override
public WriteStatus getWriteStatus() { public WriteStatus getWriteStatus() {
return writeStatus; return writeStatus;
} }
private Writer createLogWriter(Optional<FileSlice> fileSlice, String baseCommitTime)
throws IOException, InterruptedException {
return HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(
fileSlice.get().getLogFiles().map(logFile -> logFile.getLogVersion())
.max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
.withSizeThreshold(config.getLogFileMaxSize()).withFs(fs)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
}
private void writeToBuffer(HoodieRecord<T> record) {
// update the new location of the record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
if (indexedRecord.isPresent()) {
recordList.add(indexedRecord.get());
} else {
keysToDelete.add(record.getRecordKey());
}
numberOfRecords++;
}
/**
* Checks if the number of records have reached the set threshold and then flushes the records to disk
*/
private void flushToDiskIfRequired(HoodieRecord record) {
// Append if max number of records reached to achieve block size
if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) {
// Recompute averageRecordSize before writing a new block and update existing value with
// avg of new and old
logger.info("AvgRecordSize => " + averageRecordSize);
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2;
doAppend(header);
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
numberOfRecords = 0;
}
}
} }

View File

@@ -38,7 +38,7 @@ import org.apache.log4j.Logger;
/** /**
* Cleaner is responsible for garbage collecting older files in a given partition path, such that * Cleaner is responsible for garbage collecting older files in a given partition path, such that
* <p> 1) It provides sufficient time for existing queries running on older versions, to finish <p> * <p> 1) It provides sufficient time for existing queries running on older versions, to close <p>
* 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done * 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done
* based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata} * based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
*/ */

View File

@@ -52,6 +52,7 @@ import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
/** /**
* Archiver to bound the growth of <action>.commit files * Archiver to bound the growth of <action>.commit files
@@ -99,9 +100,9 @@ public class HoodieCommitArchiveLog {
/** /**
* Check if commits need to be archived. If yes, archive commits. * Check if commits need to be archived. If yes, archive commits.
*/ */
public boolean archiveIfRequired() { public boolean archiveIfRequired(final JavaSparkContext jsc) {
try { try {
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList()); List<HoodieInstant> instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList());
boolean success = true; boolean success = true;
if (instantsToArchive.iterator().hasNext()) { if (instantsToArchive.iterator().hasNext()) {
this.writer = openWriter(); this.writer = openWriter();
@@ -117,13 +118,13 @@ public class HoodieCommitArchiveLog {
} }
} }
private Stream<HoodieInstant> getInstantsToArchive() { private Stream<HoodieInstant> getInstantsToArchive(JavaSparkContext jsc) {
// TODO : rename to max/minInstantsToKeep // TODO : rename to max/minInstantsToKeep
int maxCommitsToKeep = config.getMaxCommitsToKeep(); int maxCommitsToKeep = config.getMaxCommitsToKeep();
int minCommitsToKeep = config.getMinCommitsToKeep(); int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
// GroupBy each action and limit each action timeline to maxCommitsToKeep // GroupBy each action and limit each action timeline to maxCommitsToKeep
// TODO: Handle ROLLBACK_ACTION in future // TODO: Handle ROLLBACK_ACTION in future

View File

@@ -31,8 +31,8 @@ import com.uber.hoodie.io.storage.HoodieStorageWriter;
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory; import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
import com.uber.hoodie.table.HoodieTable; import com.uber.hoodie.table.HoodieTable;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator;
import java.util.Optional; import java.util.Optional;
import java.util.UUID;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
@@ -49,12 +49,13 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
private Path tempPath = null; private Path tempPath = null;
private long recordsWritten = 0; private long recordsWritten = 0;
private long recordsDeleted = 0; private long recordsDeleted = 0;
private Iterator<HoodieRecord<T>> recordIterator;
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable, public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
String partitionPath) { String partitionPath, String fileId) {
super(config, commitTime, hoodieTable); super(config, commitTime, hoodieTable);
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName()); this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
status.setFileId(UUID.randomUUID().toString()); status.setFileId(fileId);
status.setPartitionPath(partitionPath); status.setPartitionPath(partitionPath);
final int sparkPartitionId = TaskContext.getPartitionId(); final int sparkPartitionId = TaskContext.getPartitionId();
@@ -77,12 +78,13 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
logger.info("New InsertHandle for partition :" + partitionPath); logger.info("New InsertHandle for partition :" + partitionPath);
} }
/** public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
* Determines whether we can accept the incoming records, into the current file, depending on String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordIterator) {
* <p> this(config, commitTime, hoodieTable, partitionPath, fileId);
* - Whether it belongs to the same partitionPath as existing records - Whether the current file this.recordIterator = recordIterator;
* written bytes lt max file size }
*/
@Override
public boolean canWrite(HoodieRecord record) { public boolean canWrite(HoodieRecord record) {
return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath()); return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath());
} }
@@ -114,9 +116,30 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
} }
} }
/**
* Writes all records passed
*/
public void write() {
try {
while (recordIterator.hasNext()) {
HoodieRecord<T> record = recordIterator.next();
write(record, record.getData().getInsertValue(schema));
}
} catch (IOException io) {
throw new HoodieInsertException(
"Failed to insert records for path " + getStorageWriterPath(), io);
}
}
@Override
public WriteStatus getWriteStatus() {
return status;
}
/** /**
* Performs actions to durably, persist the current changes and returns a WriteStatus object * Performs actions to durably, persist the current changes and returns a WriteStatus object
*/ */
@Override
public WriteStatus close() { public WriteStatus close() {
logger.info("Closing the file " + status.getFileId() + " as we are done with all the records " logger.info("Closing the file " + status.getFileId() + " as we are done with all the records "
+ recordsWritten); + recordsWritten);

View File

@@ -16,6 +16,8 @@
package com.uber.hoodie.io; package com.uber.hoodie.io;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.HoodieTimeline;
@@ -26,7 +28,9 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable; import com.uber.hoodie.table.HoodieTable;
import java.io.IOException; import java.io.IOException;
import java.util.Optional;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@@ -103,4 +107,25 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
public Schema getSchema() { public Schema getSchema() {
return schema; return schema;
} }
/**
* Determines whether we can accept the incoming records, into the current file, depending on
* <p>
* - Whether it belongs to the same partitionPath as existing records - Whether the current file
* written bytes lt max file size
*/
public boolean canWrite(HoodieRecord record) {
return false;
}
/**
* Perform the actual writing of the given record into the backing file.
*/
public void write(HoodieRecord record, Optional<IndexedRecord> insertValue) {
// NO_OP
}
public abstract WriteStatus close();
public abstract WriteStatus getWriteStatus();
} }

View File

@@ -243,7 +243,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
} }
} }
public void close() { @Override
public WriteStatus close() {
try { try {
// write out any pending records (this can happen when inserts are turned into updates) // write out any pending records (this can happen when inserts are turned into updates)
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator(); Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
@@ -269,6 +270,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
RuntimeStats runtimeStats = new RuntimeStats(); RuntimeStats runtimeStats = new RuntimeStats();
runtimeStats.setTotalUpsertTime(timer.endTimer()); runtimeStats.setTotalUpsertTime(timer.endTimer());
writeStatus.getStat().setRuntimeStats(runtimeStats); writeStatus.getStat().setRuntimeStats(runtimeStats);
return writeStatus;
} catch (IOException e) { } catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e); throw new HoodieUpsertException("Failed to close UpdateHandle", e);
} }
@@ -283,6 +285,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
return (this.tempPath == null) ? this.newFilePath : this.tempPath; return (this.tempPath == null) ? this.newFilePath : this.tempPath;
} }
@Override
public WriteStatus getWriteStatus() { public WriteStatus getWriteStatus() {
return writeStatus; return writeStatus;
} }

View File

@@ -18,11 +18,13 @@ package com.uber.hoodie.io.compact;
import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy; import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import java.io.Serializable; import java.io.Serializable;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
/** /**
@@ -33,10 +35,10 @@ import java.util.stream.Collectors;
*/ */
public class CompactionOperation implements Serializable { public class CompactionOperation implements Serializable {
private String dataFileCommitTime; private Optional<String> dataFileCommitTime;
private long dataFileSize; private Optional<Long> dataFileSize;
private List<String> deltaFilePaths; private List<String> deltaFilePaths;
private String dataFilePath; private Optional<String> dataFilePath;
private String fileId; private String fileId;
private String partitionPath; private String partitionPath;
private Map<String, Object> metrics; private Map<String, Object> metrics;
@@ -46,24 +48,32 @@ public class CompactionOperation implements Serializable {
public CompactionOperation() { public CompactionOperation() {
} }
public CompactionOperation(HoodieDataFile dataFile, String partitionPath, public CompactionOperation(Optional<HoodieDataFile> dataFile, String partitionPath,
List<HoodieLogFile> logFiles, HoodieWriteConfig writeConfig) { List<HoodieLogFile> logFiles, HoodieWriteConfig writeConfig) {
this.dataFilePath = dataFile.getPath(); if (dataFile.isPresent()) {
this.fileId = dataFile.getFileId(); this.dataFilePath = Optional.of(dataFile.get().getPath());
this.fileId = dataFile.get().getFileId();
this.dataFileCommitTime = Optional.of(dataFile.get().getCommitTime());
this.dataFileSize = Optional.of(dataFile.get().getFileSize());
} else {
assert logFiles.size() > 0;
this.dataFilePath = Optional.empty();
this.fileId = FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath());
this.dataFileCommitTime = Optional.empty();
this.dataFileSize = Optional.empty();
}
this.partitionPath = partitionPath; this.partitionPath = partitionPath;
this.dataFileCommitTime = dataFile.getCommitTime();
this.dataFileSize = dataFile.getFileSize();
this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()) this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString())
.collect(Collectors.toList()); .collect(Collectors.toList());
this.metrics = writeConfig.getCompactionStrategy() this.metrics = writeConfig.getCompactionStrategy()
.captureMetrics(dataFile, partitionPath, logFiles); .captureMetrics(writeConfig, dataFile, partitionPath, logFiles);
} }
public String getDataFileCommitTime() { public Optional<String> getDataFileCommitTime() {
return dataFileCommitTime; return dataFileCommitTime;
} }
public long getDataFileSize() { public Optional<Long> getDataFileSize() {
return dataFileSize; return dataFileSize;
} }
@@ -71,7 +81,7 @@ public class CompactionOperation implements Serializable {
return deltaFilePaths; return deltaFilePaths;
} }
public String getDataFilePath() { public Optional<String> getDataFilePath() {
return dataFilePath; return dataFilePath;
} }

View File

@@ -86,16 +86,19 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
private JavaRDD<WriteStatus> executeCompaction(JavaSparkContext jsc, private JavaRDD<WriteStatus> executeCompaction(JavaSparkContext jsc,
List<CompactionOperation> operations, HoodieTable hoodieTable, HoodieWriteConfig config, List<CompactionOperation> operations, HoodieTable hoodieTable, HoodieWriteConfig config,
String compactionCommitTime) throws IOException { String compactionCommitTime) throws IOException {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
// Compacting is very similar to applying updates to existing file
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
log.info("After filtering, Compacting " + operations + " files"); log.info("After filtering, Compacting " + operations + " files");
return jsc.parallelize(operations, operations.size()) return jsc.parallelize(operations, operations.size())
.map(s -> compact(hoodieTable, config, s, compactionCommitTime)) .map(s -> compact(table, metaClient, config, s, compactionCommitTime))
.flatMap(writeStatusesItr -> writeStatusesItr.iterator()); .flatMap(writeStatusesItr -> writeStatusesItr.iterator());
} }
private List<WriteStatus> compact(HoodieTable hoodieTable, HoodieWriteConfig config, private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
HoodieWriteConfig config,
CompactionOperation operation, String commitTime) throws IOException { CompactionOperation operation, String commitTime) throws IOException {
FileSystem fs = hoodieTable.getMetaClient().getFs(); FileSystem fs = metaClient.getFs();
Schema readerSchema = HoodieAvroUtils Schema readerSchema = HoodieAvroUtils
.addMetadataFields(new Schema.Parser().parse(config.getSchema())); .addMetadataFields(new Schema.Parser().parse(config.getSchema()));
@@ -107,7 +110,6 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// Load all the delta commits since the last compaction commit and get all the blocks to be // Load all the delta commits since the last compaction commit and get all the blocks to be
// loaded and load it using CompositeAvroLogReader // loaded and load it using CompositeAvroLogReader
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon. // Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
String maxInstantTime = metaClient.getActiveTimeline() String maxInstantTime = metaClient.getActiveTimeline()
.getTimelineOfActions( .getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
@@ -125,9 +127,16 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
} }
// Compacting is very similar to applying updates to existing file // Compacting is very similar to applying updates to existing file
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); Iterator<List<WriteStatus>> result;
Iterator<List<WriteStatus>> result = table // If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a
.handleUpdate(commitTime, operation.getFileId(), scanner.getRecords()); // new base parquet file.
if (operation.getDataFilePath().isPresent()) {
result = hoodieCopyOnWriteTable
.handleUpdate(commitTime, operation.getFileId(), scanner.getRecords());
} else {
result = hoodieCopyOnWriteTable
.handleInsert(commitTime, operation.getPartitionPath(), operation.getFileId(), scanner.iterator());
}
Iterable<List<WriteStatus>> resultIterable = () -> result; Iterable<List<WriteStatus>> resultIterable = () -> result;
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream) return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
.map(s -> { .map(s -> {
@@ -176,7 +185,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
.getLogVersionComparator().reversed()).collect(Collectors.toList()); .getLogVersionComparator().reversed()).collect(Collectors.toList());
totalLogFiles.add((long) logFiles.size()); totalLogFiles.add((long) logFiles.size());
totalFileSlices.add(1L); totalFileSlices.add(1L);
return new CompactionOperation(s.getDataFile().get(), partitionPath, logFiles, config); return new CompactionOperation(s.getDataFile(), partitionPath, logFiles, config);
}) })
.filter(c -> !c.getDeltaFilePaths().isEmpty()) .filter(c -> !c.getDeltaFilePaths().isEmpty())
.collect(toList()).iterator()).collect(); .collect(toList()).iterator()).collect();
@@ -195,4 +204,4 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
return operations; return operations;
} }
} }

View File

@@ -54,16 +54,19 @@ public abstract class CompactionStrategy implements Serializable {
* @param logFiles - List of log files to compact with the base file * @param logFiles - List of log files to compact with the base file
* @return Map[String, Object] - metrics captured * @return Map[String, Object] - metrics captured
*/ */
public Map<String, Object> captureMetrics(HoodieDataFile dataFile, String partitionPath, public Map<String, Object> captureMetrics(HoodieWriteConfig writeConfig, Optional<HoodieDataFile> dataFile, String
List<HoodieLogFile> logFiles) { partitionPath, List<HoodieLogFile> logFiles) {
Map<String, Object> metrics = Maps.newHashMap(); Map<String, Object> metrics = Maps.newHashMap();
Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize();
// Total size of all the log files // Total size of all the log files
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(Optional::isPresent) Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(Optional::isPresent)
.map(Optional::get).reduce((size1, size2) -> size1 + size2).orElse(0L); .map(Optional::get).reduce((size1, size2) -> size1 + size2).orElse(0L);
// Total read will be the base file + all the log files // Total read will be the base file + all the log files
Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize); Long totalIORead = FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L)
+ totalLogFileSize);
// Total write will be similar to the size of the base file // Total write will be similar to the size of the base file
Long totalIOWrite = FSUtils.getSizeInMB(dataFile.getFileSize()); Long totalIOWrite = FSUtils
.getSizeInMB(dataFile.isPresent() ? dataFile.get().getFileSize() : defaultMaxParquetFileSize);
// Total IO will the the IO for read + write // Total IO will the the IO for read + write
Long totalIO = totalIORead + totalIOWrite; Long totalIO = totalIORead + totalIOWrite;
// Save these metrics and we will use during the filter // Save these metrics and we will use during the filter

View File

@@ -39,10 +39,11 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat
private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE"; private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE";
@Override @Override
public Map<String, Object> captureMetrics(HoodieDataFile dataFile, String partitionPath, public Map<String, Object> captureMetrics(HoodieWriteConfig config, Optional<HoodieDataFile> dataFile, String
partitionPath,
List<HoodieLogFile> logFiles) { List<HoodieLogFile> logFiles) {
Map<String, Object> metrics = super.captureMetrics(dataFile, partitionPath, logFiles); Map<String, Object> metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles);
// Total size of all the log files // Total size of all the log files
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize) Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize)
.filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2) .filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2)

View File

@@ -72,7 +72,7 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
this.schema = schema; this.schema = schema;
} }
private static Configuration registerFileSystem(Path file, Configuration conf) { public static Configuration registerFileSystem(Path file, Configuration conf) {
Configuration returnConf = new Configuration(conf); Configuration returnConf = new Configuration(conf);
String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl", returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",

View File

@@ -30,18 +30,17 @@ import org.apache.parquet.hadoop.metadata.CompressionCodecName;
public class HoodieStorageWriterFactory { public class HoodieStorageWriterFactory {
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
getStorageWriter(String commitTime, Path path, HoodieTable<T> hoodieTable, String commitTime, Path path, HoodieTable<T> hoodieTable,
HoodieWriteConfig config, Schema schema) throws IOException { HoodieWriteConfig config, Schema schema) throws IOException {
//TODO - based on the metadata choose the implementation of HoodieStorageWriter //TODO - based on the metadata choose the implementation of HoodieStorageWriter
// Currently only parquet is supported // Currently only parquet is supported
return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable); return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable);
} }
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> private static <T extends HoodieRecordPayload,
newParquetStorageWriter( R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(String commitTime, Path path,
String commitTime, Path path, HoodieWriteConfig config, Schema schema, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable) throws IOException {
HoodieTable hoodieTable) throws IOException {
BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(), BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(),
config.getBloomFilterFPP()); config.getBloomFilterFPP());
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(

View File

@@ -40,10 +40,11 @@ import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.LazyInsertIterable; import com.uber.hoodie.func.CopyOnWriteLazyInsertIterable;
import com.uber.hoodie.func.ParquetReaderIterator; import com.uber.hoodie.func.ParquetReaderIterator;
import com.uber.hoodie.func.SparkBoundedInMemoryExecutor; import com.uber.hoodie.func.SparkBoundedInMemoryExecutor;
import com.uber.hoodie.io.HoodieCleanHelper; import com.uber.hoodie.io.HoodieCleanHelper;
import com.uber.hoodie.io.HoodieCreateHandle;
import com.uber.hoodie.io.HoodieMergeHandle; import com.uber.hoodie.io.HoodieMergeHandle;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
@@ -64,6 +65,7 @@ import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroParquetReader; import org.apache.parquet.avro.AvroParquetReader;
@@ -90,8 +92,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class); private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class);
public HoodieCopyOnWriteTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { public HoodieCopyOnWriteTable(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, metaClient); super(config, jsc);
} }
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String,
@@ -225,7 +227,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
public Iterator<List<WriteStatus>> handleInsert(String commitTime, public Iterator<List<WriteStatus>> handleInsert(String commitTime,
Iterator<HoodieRecord<T>> recordItr) throws Exception { Iterator<HoodieRecord<T>> recordItr) throws Exception {
return new LazyInsertIterable<>(recordItr, config, commitTime, this); return new CopyOnWriteLazyInsertIterable<>(recordItr, config, commitTime, this);
}
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String partitionPath, String fileId,
Iterator<HoodieRecord<T>> recordItr) {
HoodieCreateHandle createHandle = new HoodieCreateHandle(config, commitTime, this, partitionPath, fileId,
recordItr);
createHandle.write();
return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator();
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@@ -289,17 +299,29 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
*/ */
protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits) protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits)
throws IOException { throws IOException {
Map<FileStatus, Boolean> results = Maps.newHashMap();
// PathFilter to get all parquet files and log files that need to be deleted
PathFilter filter = (path) -> {
if (path.toString().contains(".parquet")) {
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commits.contains(fileCommitTime);
}
return false;
};
deleteCleanedFiles(results, partitionPath, filter);
return results;
}
/**
* Common method used for cleaning out parquet files under a partition path during rollback of a
* set of commits
*/
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, String partitionPath,
PathFilter filter)
throws IOException {
logger.info("Cleaning path " + partitionPath); logger.info("Cleaning path " + partitionPath);
FileSystem fs = getMetaClient().getFs(); FileSystem fs = getMetaClient().getFs();
FileStatus[] toBeDeleted = fs FileStatus[] toBeDeleted = fs.listStatus(new Path(config.getBasePath(), partitionPath), filter);
.listStatus(new Path(config.getBasePath(), partitionPath), path -> {
if (!path.toString().contains(".parquet")) {
return false;
}
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commits.contains(fileCommitTime);
});
Map<FileStatus, Boolean> results = Maps.newHashMap();
for (FileStatus file : toBeDeleted) { for (FileStatus file : toBeDeleted) {
boolean success = fs.delete(file.getPath(), false); boolean success = fs.delete(file.getPath(), false);
results.put(file, success); results.put(file, success);
@@ -311,7 +333,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
@Override @Override
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits) public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
throws IOException { throws IOException {
String actionType = this.getCommitActionType(); String actionType = metaClient.getCommitActionType();
HoodieActiveTimeline activeTimeline = this.getActiveTimeline(); HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
List<String> inflights = this.getInflightCommitTimeline().getInstants() List<String> inflights = this.getInflightCommitTimeline().getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList()); .map(HoodieInstant::getTimestamp).collect(Collectors.toList());

View File

@@ -27,11 +27,12 @@ import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat; import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock; import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
@@ -39,6 +40,7 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.MergeOnReadLazyInsertIterable;
import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieAppendHandle; import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
@@ -55,6 +57,7 @@ import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
import org.apache.spark.Partitioner; import org.apache.spark.Partitioner;
@@ -78,8 +81,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
// UpsertPartitioner for MergeOnRead table type // UpsertPartitioner for MergeOnRead table type
private MergeOnReadUpsertPartitioner mergeOnReadUpsertPartitioner; private MergeOnReadUpsertPartitioner mergeOnReadUpsertPartitioner;
public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { public HoodieMergeOnReadTable(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, metaClient); super(config, jsc);
} }
@Override @Override
@@ -110,6 +113,17 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
} }
} }
@Override
public Iterator<List<WriteStatus>> handleInsert(String commitTime,
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);
} else {
return super.handleInsert(commitTime, recordItr);
}
}
@Override @Override
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionCommitTime) { public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionCommitTime) {
logger.info("Checking if compaction needs to be run on " + config.getBasePath()); logger.info("Checking if compaction needs to be run on " + config.getBasePath());
@@ -166,6 +180,20 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
return commits.stream().map(commit -> { return commits.stream().map(commit -> {
HoodieInstant instant = commitsAndCompactions.get(commit); HoodieInstant instant = commitsAndCompactions.get(commit);
HoodieRollbackStat hoodieRollbackStats = null; HoodieRollbackStat hoodieRollbackStats = null;
// Need to put the path filter here since Filter is not serializable
// PathFilter to get all parquet files and log files that need to be deleted
PathFilter filter = (path) -> {
if (path.toString().contains(".parquet")) {
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commits.contains(fileCommitTime);
} else if (path.toString().contains(".log")) {
// Since the baseCommitTime is the only commit for new log files, it's okay here
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
return commits.contains(fileCommitTime);
}
return false;
};
switch (instant.getAction()) { switch (instant.getAction()) {
case HoodieTimeline.COMMIT_ACTION: case HoodieTimeline.COMMIT_ACTION:
try { try {
@@ -180,18 +208,18 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
case HoodieTimeline.DELTA_COMMIT_ACTION: case HoodieTimeline.DELTA_COMMIT_ACTION:
try { try {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
this.getCommitTimeline().getInstantDetails( metaClient.getCommitTimeline().getInstantDetails(
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())) new HoodieInstant(true, instant.getAction(), instant.getTimestamp()))
.get()); .get());
// read commit file and (either append delete blocks or delete file) // read commit file and (either append delete blocks or delete file)
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>(); final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>(); Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
// we do not know fileIds for inserts (first inserts are parquet files), delete // In case all data was inserts and the commit failed, there is no partition stats
// all parquet files for the corresponding failed commit, if present (same as COW) if (commitMetadata.getPartitionToWriteStats().size() == 0) {
filesToDeletedStatus = super super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
.deleteCleanedFiles(partitionPath, Arrays.asList(commit)); }
// append rollback blocks for updates // append rollback blocks for updates
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
@@ -201,15 +229,31 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseCommitTime)) : null; .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseCommitTime)) : null;
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
.filter(wStat -> { .filter(wStat -> {
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT if (wStat != null
&& wStat.getPrevCommit() != null; && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT
}).forEach(wStat -> { && wStat.getPrevCommit() != null) {
HoodieLogFormat.Writer writer = null; return true;
}
// we do not know fileIds for inserts (first inserts are either log files or parquet files),
// delete all files for the corresponding failed commit, if present (same as COW)
try {
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
} catch (IOException io) {
throw new UncheckedIOException(io);
}
return false;
})
.forEach(wStat -> {
Writer writer = null;
String baseCommitTime = wStat.getPrevCommit(); String baseCommitTime = wStat.getPrevCommit();
if (hoodieIndex.isGlobal()) { if (hoodieIndex.isGlobal()) {
baseCommitTime = fileIdToLatestCommitTimeMap.get(wStat.getFileId()); baseCommitTime = fileIdToLatestCommitTimeMap.get(wStat.getFileId());
} }
try { try {
// TODO : wStat.getPrevCommit() might not give the right commit time in the following
// scenario if a compaction was scheduled, the new commitTime will be used to write the
// new log file. In this case, the commit time for the log file is the
// getBaseCommitTime()
writer = HoodieLogFormat.newWriterBuilder().onParentPath( writer = HoodieLogFormat.newWriterBuilder().onParentPath(
new Path(this.getMetaClient().getBasePath(), partitionPath)) new Path(this.getMetaClient().getBasePath(), partitionPath))
.withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime) .withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime)
@@ -217,15 +261,15 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
Long numRollbackBlocks = 0L; Long numRollbackBlocks = 0L;
// generate metadata // generate metadata
Map<HoodieLogBlock.HeaderMetadataType, String> header = Map<HeaderMetadataType, String> header =
Maps.newHashMap(); Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, header.put(HeaderMetadataType.INSTANT_TIME,
metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, header.put(HeaderMetadataType.TARGET_INSTANT_TIME,
commit); commit);
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String
.valueOf( .valueOf(
HoodieCommandBlock.HoodieCommandBlockTypeEnum HoodieCommandBlockTypeEnum
.ROLLBACK_PREVIOUS_BLOCK .ROLLBACK_PREVIOUS_BLOCK
.ordinal())); .ordinal()));
// if update belongs to an existing log file // if update belongs to an existing log file
@@ -263,7 +307,6 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
commitsAndCompactions.entrySet().stream().map( commitsAndCompactions.entrySet().stream().map(
entry -> new HoodieInstant(true, entry.getValue().getAction(), entry -> new HoodieInstant(true, entry.getValue().getAction(),
entry.getValue().getTimestamp())).forEach(this.getActiveTimeline()::deleteInflight); entry.getValue().getTimestamp())).forEach(this.getActiveTimeline()::deleteInflight);
logger logger
.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime)); .debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
@@ -298,6 +341,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
if (!commitTimeline.empty()) { if (!commitTimeline.empty()) {
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
// find smallest file in partition and append to it // find smallest file in partition and append to it
// TODO - check if index.isglobal then small files are log files too
Optional<FileSlice> smallFileSlice = getRTFileSystemView() Optional<FileSlice> smallFileSlice = getRTFileSystemView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter( .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter(
fileSlice -> fileSlice.getLogFiles().count() < 1 fileSlice -> fileSlice.getLogFiles().count() < 1
@@ -327,4 +372,5 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
} }
} }

View File

@@ -31,9 +31,9 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieSavepointException; import com.uber.hoodie.exception.HoodieSavepointException;
import com.uber.hoodie.index.HoodieIndex;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.util.Iterator; import java.util.Iterator;
@@ -54,19 +54,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
protected final HoodieWriteConfig config; protected final HoodieWriteConfig config;
protected final HoodieTableMetaClient metaClient; protected final HoodieTableMetaClient metaClient;
protected final HoodieIndex<T> index;
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc) {
this.config = config; this.config = config;
this.metaClient = metaClient; this.metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true);
this.index = HoodieIndex.createIndex(config, jsc);
} }
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable( public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
HoodieTableMetaClient metaClient, HoodieWriteConfig config) { HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) {
switch (metaClient.getTableType()) { switch (metaClient.getTableType()) {
case COPY_ON_WRITE: case COPY_ON_WRITE:
return new HoodieCopyOnWriteTable<>(config, metaClient); return new HoodieCopyOnWriteTable<>(config, jsc);
case MERGE_ON_READ: case MERGE_ON_READ:
return new HoodieMergeOnReadTable<>(config, metaClient); return new HoodieMergeOnReadTable<>(config, jsc);
default: default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
} }
@@ -124,21 +126,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
* Get the completed (commit + compaction) view of the file system for this table * Get the completed (commit + compaction) view of the file system for this table
*/ */
public TableFileSystemView getCompletedFileSystemView() { public TableFileSystemView getCompletedFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCommitsTimeline()); return new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline());
} }
/** /**
* Get only the completed (no-inflights) commit timeline * Get only the completed (no-inflights) commit timeline
*/ */
public HoodieTimeline getCompletedCommitTimeline() { public HoodieTimeline getCompletedCommitTimeline() {
return getCommitsTimeline().filterCompletedInstants(); return metaClient.getCommitsTimeline().filterCompletedInstants();
} }
/** /**
* Get only the inflights (no-completed) commit timeline * Get only the inflights (no-completed) commit timeline
*/ */
public HoodieTimeline getInflightCommitTimeline() { public HoodieTimeline getInflightCommitTimeline() {
return getCommitsTimeline().filterInflights(); return metaClient.getCommitsTimeline().filterInflights();
} }
/** /**
@@ -190,49 +192,10 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
} }
/** /**
* Get the commit timeline visible for this table * Return the index
*/ */
public HoodieTimeline getCommitsTimeline() { public HoodieIndex<T> getIndex() {
switch (metaClient.getTableType()) { return index;
case COPY_ON_WRITE:
return getActiveTimeline().getCommitTimeline();
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits
// Include commit action to be able to start doing a MOR over a COW dataset - no
// migration required
return getActiveTimeline().getCommitsTimeline();
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
/**
* Get the compacted commit timeline visible for this table
*/
public HoodieTimeline getCommitTimeline() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits in tagging
return getActiveTimeline().getCommitTimeline();
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
/**
* Gets the commit action type
*/
public String getCommitActionType() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return HoodieActiveTimeline.COMMIT_ACTION;
case MERGE_ON_READ:
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
default:
throw new HoodieCommitException(
"Could not commit on unknown storage type " + metaClient.getTableType());
}
} }
/** /**

View File

@@ -76,10 +76,10 @@ public class TestCleaner extends TestHoodieClientBase {
/** /**
* Helper method to do first batch of insert for clean by versions/commits tests * Helper method to do first batch of insert for clean by versions/commits tests
* *
* @param cfg Hoodie Write Config * @param cfg Hoodie Write Config
* @param client Hoodie Client * @param client Hoodie Client
* @param recordGenFunction Function to generate records for insertion * @param recordGenFunction Function to generate records for insertion
* @param insertFn Insertion API for testing * @param insertFn Insertion API for testing
* @throws Exception in case of error * @throws Exception in case of error
*/ */
private void insertFirstBigBatchForClientCleanerTest( private void insertFirstBigBatchForClientCleanerTest(
@@ -93,7 +93,6 @@ public class TestCleaner extends TestHoodieClientBase {
* (this is basically same as insert part of upsert, just adding it here so we can * (this is basically same as insert part of upsert, just adding it here so we can
* catch breakages in insert(), if the implementation diverges.) * catch breakages in insert(), if the implementation diverges.)
*/ */
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
String newCommitTime = client.startCommit(); String newCommitTime = client.startCommit();
List<HoodieRecord> records = recordGenFunction.apply(newCommitTime, BIG_BATCH_INSERT_SIZE); List<HoodieRecord> records = recordGenFunction.apply(newCommitTime, BIG_BATCH_INSERT_SIZE);
@@ -108,7 +107,7 @@ public class TestCleaner extends TestHoodieClientBase {
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
// Should have 100 records in table (check using Index), all in locations marked at commit // Should have 100 records in table (check using Index), all in locations marked at commit
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
assertFalse(table.getCompletedCommitTimeline().empty()); assertFalse(table.getCompletedCommitTimeline().empty());
String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp();
@@ -116,7 +115,8 @@ public class TestCleaner extends TestHoodieClientBase {
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", commitTime,
table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp()); table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect(); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table).collect();
checkTaggedRecords(taggedRecords, newCommitTime); checkTaggedRecords(taggedRecords, newCommitTime);
} }
@@ -158,10 +158,10 @@ public class TestCleaner extends TestHoodieClientBase {
/** /**
* Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective * Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective
* *
* @param insertFn Insert API to be tested * @param insertFn Insert API to be tested
* @param upsertFn Upsert API to be tested * @param upsertFn Upsert API to be tested
* @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during
* record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs)
* @throws Exception in case of errors * @throws Exception in case of errors
*/ */
private void testInsertAndCleanByVersions( private void testInsertAndCleanByVersions(
@@ -198,8 +198,8 @@ public class TestCleaner extends TestHoodieClientBase {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig()); HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
HoodieTimeline timeline = table.getCommitsTimeline(); HoodieTimeline timeline = metadata.getCommitsTimeline();
TableFileSystemView fsView = table.getFileSystemView(); TableFileSystemView fsView = table.getFileSystemView();
// Need to ensure the following // Need to ensure the following
@@ -280,10 +280,10 @@ public class TestCleaner extends TestHoodieClientBase {
/** /**
* Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective * Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective
* *
* @param insertFn Insert API to be tested * @param insertFn Insert API to be tested
* @param upsertFn Upsert API to be tested * @param upsertFn Upsert API to be tested
* @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during
* record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs)
* @throws Exception in case of errors * @throws Exception in case of errors
*/ */
private void testInsertAndCleanByCommits( private void testInsertAndCleanByCommits(
@@ -318,7 +318,7 @@ public class TestCleaner extends TestHoodieClientBase {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg); HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg, jsc);
HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline(); HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline();
Optional<HoodieInstant> earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); Optional<HoodieInstant> earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
Set<HoodieInstant> acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet()); Set<HoodieInstant> acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet());
@@ -365,7 +365,8 @@ public class TestCleaner extends TestHoodieClientBase {
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "000"); String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "000");
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000"); String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000");
HoodieTable table = HoodieTable.getHoodieTable( HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc); List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
assertEquals("Must not clean any files", 0, assertEquals("Must not clean any files", 0,
@@ -377,7 +378,8 @@ public class TestCleaner extends TestHoodieClientBase {
// make next commit, with 1 insert & 1 update per partition // make next commit, with 1 insert & 1 update per partition
HoodieTestUtils.createCommitFiles(basePath, "001"); HoodieTestUtils.createCommitFiles(basePath, "001");
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), config); table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), config,
jsc);
String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001"); // insert String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001"); // insert
String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert
@@ -397,7 +399,7 @@ public class TestCleaner extends TestHoodieClientBase {
// make next commit, with 2 updates to existing files, and 1 insert // make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "002"); HoodieTestUtils.createCommitFiles(basePath, "002");
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config, jsc);
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update
@@ -452,7 +454,7 @@ public class TestCleaner extends TestHoodieClientBase {
// make 1 compaction commit // make 1 compaction commit
HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001");
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc); List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
assertEquals("Must clean three files, one parquet and 2 log files", 3, assertEquals("Must clean three files, one parquet and 2 log files", 3,
getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size());
@@ -479,7 +481,8 @@ public class TestCleaner extends TestHoodieClientBase {
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000"); String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000");
HoodieTable table = HoodieTable.getHoodieTable( HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc); List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
assertEquals("Must not clean any files", 0, assertEquals("Must not clean any files", 0,
@@ -492,7 +495,7 @@ public class TestCleaner extends TestHoodieClientBase {
// make next commit, with 1 insert & 1 update per partition // make next commit, with 1 insert & 1 update per partition
HoodieTestUtils.createCommitFiles(basePath, "001"); HoodieTestUtils.createCommitFiles(basePath, "001");
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config, jsc);
String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001"); // insert String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001"); // insert
String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert
@@ -512,7 +515,7 @@ public class TestCleaner extends TestHoodieClientBase {
// make next commit, with 2 updates to existing files, and 1 insert // make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "002"); HoodieTestUtils.createCommitFiles(basePath, "002");
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config, jsc);
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "002", file2P0C1); // update
@@ -527,7 +530,7 @@ public class TestCleaner extends TestHoodieClientBase {
// make next commit, with 2 updates to existing files, and 1 insert // make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "003"); HoodieTestUtils.createCommitFiles(basePath, "003");
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config, jsc);
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file2P0C1); // update HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "003", file2P0C1); // update
@@ -568,14 +571,15 @@ public class TestCleaner extends TestHoodieClientBase {
.withUseTempFolderCopyOnWriteForCreate(false) .withUseTempFolderCopyOnWriteForCreate(false)
.withUseTempFolderCopyOnWriteForMerge(false).build(); .withUseTempFolderCopyOnWriteForMerge(false).build();
HoodieTable table = HoodieTable.getHoodieTable( HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
table.rollback(jsc, Collections.emptyList()); table.rollback(jsc, Collections.emptyList());
assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles());
config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true) config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true)
.withUseTempFolderCopyOnWriteForMerge(false).build(); .withUseTempFolderCopyOnWriteForMerge(false).build();
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config, jsc);
table.rollback(jsc, Collections.emptyList()); table.rollback(jsc, Collections.emptyList());
assertEquals("All temp files are deleted.", 0, getTotalTempFiles()); assertEquals("All temp files are deleted.", 0, getTotalTempFiles());
} }
@@ -595,7 +599,8 @@ public class TestCleaner extends TestHoodieClientBase {
HoodieTestUtils.createCommitFiles(basePath, "000"); HoodieTestUtils.createCommitFiles(basePath, "000");
HoodieTable table = HoodieTable.getHoodieTable( HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc); List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty()); assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty());
@@ -655,7 +660,8 @@ public class TestCleaner extends TestHoodieClientBase {
updateAllFilesInPartition(filesP2C0, DEFAULT_THIRD_PARTITION_PATH, "003"); updateAllFilesInPartition(filesP2C0, DEFAULT_THIRD_PARTITION_PATH, "003");
HoodieTable table = HoodieTable.getHoodieTable( HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc); List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
assertEquals(100, getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); assertEquals(100, getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size());
@@ -677,7 +683,7 @@ public class TestCleaner extends TestHoodieClientBase {
* Utility method to create temporary data files * Utility method to create temporary data files
* *
* @param commitTime Commit Timestamp * @param commitTime Commit Timestamp
* @param numFiles Number for files to be generated * @param numFiles Number for files to be generated
* @return generated files * @return generated files
* @throws IOException in case of error * @throws IOException in case of error
*/ */

View File

@@ -96,7 +96,7 @@ public class TestClientRollback extends TestHoodieClientBase {
List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(),
getConfig().shouldAssumeDatePartitioning()); getConfig().shouldAssumeDatePartitioning());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView();
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> { List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
@@ -121,7 +121,7 @@ public class TestClientRollback extends TestHoodieClientBase {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig()); table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView(); final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> { dataFiles = partitionPaths.stream().flatMap(s -> {
@@ -142,7 +142,7 @@ public class TestClientRollback extends TestHoodieClientBase {
client.rollbackToSavepoint(savepoint.getTimestamp()); client.rollbackToSavepoint(savepoint.getTimestamp());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig()); table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView(); final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> { dataFiles = partitionPaths.stream().flatMap(s -> {
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));

View File

@@ -184,9 +184,9 @@ public class TestHoodieClientBase implements Serializable {
final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc); final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc);
List<HoodieRecord> records = recordGenFunction.apply(commit, numRecords); List<HoodieRecord> records = recordGenFunction.apply(commit, numRecords);
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true); final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
HoodieTable.getHoodieTable(metaClient, writeConfig); HoodieTable table = HoodieTable.getHoodieTable(metaClient, writeConfig, jsc);
JavaRDD<HoodieRecord> taggedRecords = JavaRDD<HoodieRecord> taggedRecords =
index.tagLocation(jsc.parallelize(records, 1), HoodieTable.getHoodieTable(metaClient, writeConfig)); index.tagLocation(jsc.parallelize(records, 1), jsc, table);
return taggedRecords.collect(); return taggedRecords.collect();
}; };
} }
@@ -348,7 +348,7 @@ public class TestHoodieClientBase implements Serializable {
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
} }
assertEquals("Must contain " + expTotalRecords + " records", expTotalRecords, assertEquals("Must contain " + expTotalRecords + " records", expTotalRecords,
HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count());
// Check that the incremental consumption from prevCommitTime // Check that the incremental consumption from prevCommitTime
assertEquals("Incremental consumption from " + prevCommitTime assertEquals("Incremental consumption from " + prevCommitTime

View File

@@ -415,7 +415,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
assertEquals("2 files needs to be committed.", 2, statuses.size()); assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView(); TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView();
List<HoodieDataFile> files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) List<HoodieDataFile> files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
.collect(Collectors.toList()); .collect(Collectors.toList());
@@ -519,7 +519,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
assertEquals("2 files needs to be committed.", 2, statuses.size()); assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
List<HoodieDataFile> files = table.getROFileSystemView() List<HoodieDataFile> files = table.getROFileSystemView()
.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) .getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
.collect(Collectors.toList()); .collect(Collectors.toList());
@@ -544,7 +544,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
String commitTime = "000"; String commitTime = "000";
client.startCommitWithTime(commitTime); client.startCommitWithTime(commitTime);
@@ -559,9 +559,9 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
HoodieTestUtils.doesCommitExist(basePath, commitTime)); HoodieTestUtils.doesCommitExist(basePath, commitTime));
// Get parquet file paths from commit metadata // Get parquet file paths from commit metadata
String actionType = table.getCommitActionType(); String actionType = metaClient.getCommitActionType();
HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime); HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime);
HoodieTimeline commitTimeline = table.getCommitTimeline().filterCompletedInstants(); HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(commitInstant).get()); .fromBytes(commitTimeline.getInstantDetails(commitInstant).get());
String basePath = table.getMetaClient().getBasePath(); String basePath = table.getMetaClient().getBasePath();

View File

@@ -28,7 +28,6 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.table.HoodieTable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
@@ -43,6 +42,7 @@ import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row; import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.SQLContext;
@@ -152,14 +152,17 @@ public class HoodieClientTestUtils {
/** /**
* Reads the paths under the a hoodie dataset out as a DataFrame * Reads the paths under the a hoodie dataset out as a DataFrame
*/ */
public static Dataset<Row> read(String basePath, SQLContext sqlContext, FileSystem fs, String... paths) { public static Dataset<Row> read(JavaSparkContext jsc, String basePath, SQLContext
sqlContext,
FileSystem
fs, String...
paths) {
List<String> filteredPaths = new ArrayList<>(); List<String> filteredPaths = new ArrayList<>();
try { try {
HoodieTable hoodieTable = HoodieTable HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null);
for (String path : paths) { for (String path : paths) {
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView( TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(
hoodieTable.getMetaClient(), hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path))); metaClient, metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path)));
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(Collectors.toList()); List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(Collectors.toList());
for (HoodieDataFile file : latestFiles) { for (HoodieDataFile file : latestFiles) {
filteredPaths.add(file.getPath()); filteredPaths.add(file.getPath());

View File

@@ -213,10 +213,10 @@ public class HoodieTestDataGenerator {
/** /**
* Generates new updates, randomly distributed across the keys above. There can be duplicates within the returned list * Generates new updates, randomly distributed across the keys above. There can be duplicates within the returned list
*
* @param commitTime Commit Timestamp * @param commitTime Commit Timestamp
* @param n Number of updates (including dups) * @param n Number of updates (including dups)
* @return list of hoodie record updates * @return list of hoodie record updates
* @throws IOException
*/ */
public List<HoodieRecord> generateUpdates(String commitTime, Integer n) throws IOException { public List<HoodieRecord> generateUpdates(String commitTime, Integer n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>(); List<HoodieRecord> updates = new ArrayList<>();
@@ -230,10 +230,10 @@ public class HoodieTestDataGenerator {
/** /**
* Generates deduped updates of keys previously inserted, randomly distributed across the keys above. * Generates deduped updates of keys previously inserted, randomly distributed across the keys above.
*
* @param commitTime Commit Timestamp * @param commitTime Commit Timestamp
* @param n Number of unique records * @param n Number of unique records
* @return list of hoodie record updates * @return list of hoodie record updates
* @throws IOException
*/ */
public List<HoodieRecord> generateUniqueUpdates(String commitTime, Integer n) throws IOException { public List<HoodieRecord> generateUniqueUpdates(String commitTime, Integer n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>(); List<HoodieRecord> updates = new ArrayList<>();

View File

@@ -16,7 +16,7 @@
package com.uber.hoodie.func; package com.uber.hoodie.func;
import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction; import static com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.getTransformFunction;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
@@ -38,7 +38,7 @@ public class TestBoundedInMemoryExecutor {
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
private SparkBoundedInMemoryExecutor<HoodieRecord, private SparkBoundedInMemoryExecutor<HoodieRecord,
Tuple2<HoodieRecord, Optional<IndexedRecord>>, Integer> executor = null; Tuple2<HoodieRecord, Optional<IndexedRecord>>, Integer> executor = null;
@After @After
public void afterTest() { public void afterTest() {
@@ -83,4 +83,4 @@ public class TestBoundedInMemoryExecutor {
// There should be no remaining records in the buffer // There should be no remaining records in the buffer
Assert.assertFalse(executor.isRemaining()); Assert.assertFalse(executor.isRemaining());
} }
} }

View File

@@ -16,7 +16,7 @@
package com.uber.hoodie.func; package com.uber.hoodie.func;
import static com.uber.hoodie.func.LazyInsertIterable.getTransformFunction; import static com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.getTransformFunction;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.func;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import com.uber.hoodie.WriteStatus; import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.TestRawTripPayload; import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecord;
@@ -29,11 +30,14 @@ import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieCopyOnWriteTable; import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import java.io.File;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.TemporaryFolder; import org.junit.rules.TemporaryFolder;
@@ -41,6 +45,7 @@ import org.junit.rules.TemporaryFolder;
public class TestUpdateMapFunction { public class TestUpdateMapFunction {
private String basePath = null; private String basePath = null;
private transient JavaSparkContext jsc = null;
@Before @Before
public void init() throws Exception { public void init() throws Exception {
@@ -49,6 +54,18 @@ public class TestUpdateMapFunction {
folder.create(); folder.create();
this.basePath = folder.getRoot().getAbsolutePath(); this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath); HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath);
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestUpdateMapFunction"));
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
} }
@Test @Test
@@ -56,7 +73,7 @@ public class TestUpdateMapFunction {
// Create a bunch of records with a old version of schema // Create a bunch of records with a old version of schema
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt"); HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
@@ -82,7 +99,7 @@ public class TestUpdateMapFunction {
String fileId = insertResult.next().get(0).getFileId(); String fileId = insertResult.next().get(0).getFileId();
System.out.println(fileId); System.out.println(fileId);
table = new HoodieCopyOnWriteTable(config, metaClient); table = new HoodieCopyOnWriteTable(config, jsc);
// New content with values for the newly added field // New content with values for the newly added field
recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}"; + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";

View File

@@ -26,8 +26,7 @@ import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.WriteStatus; import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableConfig;
import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieIndexConfig;
@@ -74,7 +73,6 @@ public class TestHbaseIndex {
private static String tableName = "test_table"; private static String tableName = "test_table";
private String basePath = null; private String basePath = null;
private transient FileSystem fs; private transient FileSystem fs;
private HoodieTableMetaClient metaClient;
public TestHbaseIndex() throws Exception { public TestHbaseIndex() throws Exception {
} }
@@ -117,9 +115,7 @@ public class TestHbaseIndex {
folder.create(); folder.create();
basePath = folder.getRoot().getAbsolutePath(); basePath = folder.getRoot().getAbsolutePath();
// Initialize table // Initialize table
metaClient = HoodieTableMetaClient HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
.initTableType(utility.getConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE, tableName,
HoodieTableConfig.DEFAULT_PAYLOAD_CLASS);
} }
@Test @Test
@@ -132,13 +128,14 @@ public class TestHbaseIndex {
// Load to memory // Load to memory
HoodieWriteConfig config = getConfig(); HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config, jsc); HBaseIndex index = new HBaseIndex(config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
writeClient.startCommit(); writeClient.startCommit();
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Test tagLocation without any entries in index // Test tagLocation without any entries in index
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, hoodieTable); JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
// Insert 200 records // Insert 200 records
@@ -147,14 +144,15 @@ public class TestHbaseIndex {
// Now tagLocation for these records, hbaseIndex should not tag them since it was a failed // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed
// commit // commit
javaRDD = index.tagLocation(writeRecords, hoodieTable); javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
// Now commit this & update location of records inserted and validate no errors // Now commit this & update location of records inserted and validate no errors
writeClient.commit(newCommitTime, writeStatues); writeClient.commit(newCommitTime, writeStatues);
// Now tagLocation for these records, hbaseIndex should tag them correctly // Now tagLocation for these records, hbaseIndex should tag them correctly
javaRDD = index.tagLocation(writeRecords, hoodieTable); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200);
assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200); assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200);
assertTrue(javaRDD.filter( assertTrue(javaRDD.filter(
@@ -169,14 +167,14 @@ public class TestHbaseIndex {
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
// Load to memory // Load to memory
HoodieWriteConfig config = getConfig(); HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config, jsc); HBaseIndex index = new HBaseIndex(config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
String newCommitTime = writeClient.startCommit(); String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200); List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1); JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Insert 200 records // Insert 200 records
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime); JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
@@ -184,9 +182,9 @@ public class TestHbaseIndex {
// commit this upsert // commit this upsert
writeClient.commit(newCommitTime, writeStatues); writeClient.commit(newCommitTime, writeStatues);
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Now tagLocation for these records, hbaseIndex should tag them // Now tagLocation for these records, hbaseIndex should tag them
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, hoodieTable); JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200);
// check tagged records are tagged with correct fileIds // check tagged records are tagged with correct fileIds
@@ -199,9 +197,10 @@ public class TestHbaseIndex {
// Rollback the last commit // Rollback the last commit
writeClient.rollback(newCommitTime); writeClient.rollback(newCommitTime);
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
// back commit // back commit
javaRDD = index.tagLocation(writeRecords, hoodieTable); javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0); assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0);
} }
@@ -211,7 +210,7 @@ public class TestHbaseIndex {
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
HoodieWriteConfig config = getConfig(); HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config, jsc); HBaseIndex index = new HBaseIndex(config);
// Mock hbaseConnection and related entities // Mock hbaseConnection and related entities
Connection hbaseConnection = Mockito.mock(Connection.class); Connection hbaseConnection = Mockito.mock(Connection.class);
@@ -228,15 +227,15 @@ public class TestHbaseIndex {
String newCommitTime = writeClient.startCommit(); String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250); List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1); JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Insert 250 records // Insert 250 records
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime); JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
assertNoWriteErrors(writeStatues.collect()); assertNoWriteErrors(writeStatues.collect());
// Now tagLocation for these records, hbaseIndex should tag them // Now tagLocation for these records, hbaseIndex should tag them
index.tagLocation(writeRecords, hoodieTable); index.tagLocation(writeRecords, jsc, hoodieTable);
// 3 batches should be executed given batchSize = 100 and parallelism = 1 // 3 batches should be executed given batchSize = 100 and parallelism = 1
Mockito.verify(table, times(3)).get((List<Get>) anyObject()); Mockito.verify(table, times(3)).get((List<Get>) anyObject());
@@ -248,15 +247,15 @@ public class TestHbaseIndex {
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
HoodieWriteConfig config = getConfig(); HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config, jsc); HBaseIndex index = new HBaseIndex(config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
// start a commit and generate test data // start a commit and generate test data
String newCommitTime = writeClient.startCommit(); String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250); List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1); JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Insert 200 records // Insert 200 records
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime); JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
@@ -276,7 +275,7 @@ public class TestHbaseIndex {
// Get all the files generated // Get all the files generated
int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count(); int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count();
index.updateLocation(writeStatues, hoodieTable); index.updateLocation(writeStatues, jsc, hoodieTable);
// 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated, // 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated,
// so each fileId ideally gets updates // so each fileId ideally gets updates
Mockito.verify(table, atMost(numberOfDataFileIds)).put((List<Put>) anyObject()); Mockito.verify(table, atMost(numberOfDataFileIds)).put((List<Put>) anyObject());

View File

@@ -18,27 +18,58 @@ package com.uber.hoodie.index;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.bloom.HoodieBloomIndex; import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.index.hbase.HBaseIndex; import com.uber.hoodie.index.hbase.HBaseIndex;
import java.io.File;
import java.io.IOException;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.TemporaryFolder;
public class TestHoodieIndex { public class TestHoodieIndex {
private transient JavaSparkContext jsc = null;
private String basePath = null;
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieIndex"));
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@Test @Test
public void testCreateIndex() throws Exception { public void testCreateIndex() throws Exception {
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
// Different types // Different types
HoodieWriteConfig config = clientConfigBuilder.withPath("").withIndexConfig( HoodieWriteConfig config = clientConfigBuilder.withPath(basePath).withIndexConfig(
indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()).build(); indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()).build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex); assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HBaseIndex);
config = clientConfigBuilder.withPath("") config = clientConfigBuilder.withPath(basePath)
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof InMemoryHashIndex); assertTrue(HoodieIndex.createIndex(config, jsc) instanceof InMemoryHashIndex);
config = clientConfigBuilder.withPath("") config = clientConfigBuilder.withPath(basePath)
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
assertTrue(HoodieIndex.createIndex(config, null) instanceof HoodieBloomIndex); assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HoodieBloomIndex);
} }
} }

View File

@@ -95,6 +95,16 @@ public class TestHoodieBloomIndex {
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr)); schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
} }
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@Test @Test
public void testLoadUUIDsInMemory() throws IOException { public void testLoadUUIDsInMemory() throws IOException {
// Create one RDD of hoodie record // Create one RDD of hoodie record
@@ -135,7 +145,7 @@ public class TestHoodieBloomIndex {
@Test @Test
public void testLoadInvolvedFiles() throws IOException { public void testLoadInvolvedFiles() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc); HoodieBloomIndex index = new HoodieBloomIndex(config);
// Create some partitions, and put some files // Create some partitions, and put some files
// "2016/01/21": 0 file // "2016/01/21": 0 file
@@ -171,8 +181,8 @@ public class TestHoodieBloomIndex {
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, table); List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, jsc, table);
// Still 0, as no valid commit // Still 0, as no valid commit
assertEquals(filesList.size(), 0); assertEquals(filesList.size(), 0);
@@ -181,7 +191,8 @@ public class TestHoodieBloomIndex {
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile(); new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile(); new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
filesList = index.loadInvolvedFiles(partitions, table); table = HoodieTable.getHoodieTable(metadata, config, jsc);
filesList = index.loadInvolvedFiles(partitions, jsc, table);
assertEquals(filesList.size(), 4); assertEquals(filesList.size(), 4);
// these files will not have the key ranges // these files will not have the key ranges
assertNull(filesList.get(0)._2().getMaxRecordKey()); assertNull(filesList.get(0)._2().getMaxRecordKey());
@@ -205,7 +216,7 @@ public class TestHoodieBloomIndex {
public void testRangePruning() { public void testRangePruning() {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc); HoodieBloomIndex index = new HoodieBloomIndex(config);
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>(); final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"), partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"),
@@ -290,13 +301,13 @@ public class TestHoodieBloomIndex {
// Also create the metadata and config // Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// Let's tag // Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
try { try {
bloomIndex.tagLocation(recordRDD, table); bloomIndex.tagLocation(recordRDD, jsc, table);
} catch (IllegalArgumentException e) { } catch (IllegalArgumentException e) {
fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices " + "required"); fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices " + "required");
} }
@@ -332,11 +343,11 @@ public class TestHoodieBloomIndex {
// Also create the metadata and config // Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// Let's tag // Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table); JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
// Should not find any files // Should not find any files
for (HoodieRecord record : taggedRecordRDD.collect()) { for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -350,9 +361,9 @@ public class TestHoodieBloomIndex {
// We do the tag again // We do the tag again
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metadata, config); table = HoodieTable.getHoodieTable(metadata, config, jsc);
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table); taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
// Check results // Check results
for (HoodieRecord record : taggedRecordRDD.collect()) { for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -397,11 +408,11 @@ public class TestHoodieBloomIndex {
// Also create the metadata and config // Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// Let's tag // Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table); JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table);
// Should not find any files // Should not find any files
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) { for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
@@ -415,8 +426,8 @@ public class TestHoodieBloomIndex {
// We do the tag again // We do the tag again
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metadata, config); table = HoodieTable.getHoodieTable(metadata, config, jsc);
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table); taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table);
// Check results // Check results
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) { for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
@@ -465,10 +476,10 @@ public class TestHoodieBloomIndex {
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2)); JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table); JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
// Check results // Check results
for (HoodieRecord record : taggedRecordRDD.collect()) { for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -525,14 +536,4 @@ public class TestHoodieBloomIndex {
} }
return filename; return filename;
} }
@After
public void clean() {
if (jsc != null) {
jsc.stop();
}
if (basePath != null) {
new File(basePath).delete();
}
}
} }

View File

@@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry; import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.minicluster.HdfsTestService; import com.uber.hoodie.common.minicluster.HdfsTestService;
import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieLogFile;
@@ -32,6 +33,7 @@ import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@@ -44,6 +46,8 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@@ -52,8 +56,6 @@ import org.junit.rules.TemporaryFolder;
public class TestHoodieCommitArchiveLog { public class TestHoodieCommitArchiveLog {
private String basePath;
private Configuration hadoopConf;
//NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class) //NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class)
//The implementation and gurantees of many API's differ, for example check rename(src,dst) //The implementation and gurantees of many API's differ, for example check rename(src,dst)
// We need to use DFS here instead of LocalFs since the FsDataInputStream.getWrappedStream() returns a // We need to use DFS here instead of LocalFs since the FsDataInputStream.getWrappedStream() returns a
@@ -62,6 +64,9 @@ public class TestHoodieCommitArchiveLog {
private static MiniDFSCluster dfsCluster; private static MiniDFSCluster dfsCluster;
private static DistributedFileSystem dfs; private static DistributedFileSystem dfs;
private static HdfsTestService hdfsTestService; private static HdfsTestService hdfsTestService;
private String basePath;
private Configuration hadoopConf;
private JavaSparkContext jsc = null;
@AfterClass @AfterClass
public static void cleanUp() throws Exception { public static void cleanUp() throws Exception {
@@ -91,11 +96,24 @@ public class TestHoodieCommitArchiveLog {
public void init() throws Exception { public void init() throws Exception {
TemporaryFolder folder = new TemporaryFolder(); TemporaryFolder folder = new TemporaryFolder();
folder.create(); folder.create();
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCommitArchiveLog"));
basePath = folder.getRoot().getAbsolutePath(); basePath = folder.getRoot().getAbsolutePath();
hadoopConf = dfs.getConf(); hadoopConf = dfs.getConf();
jsc.hadoopConfiguration().addResource(dfs.getConf());
dfs.mkdirs(new Path(basePath));
HoodieTestUtils.init(hadoopConf, basePath); HoodieTestUtils.init(hadoopConf, basePath);
} }
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@Test @Test
public void testArchiveEmptyDataset() throws IOException { public void testArchiveEmptyDataset() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
@@ -103,7 +121,7 @@ public class TestHoodieCommitArchiveLog {
.forTable("test-trip-table").build(); .forTable("test-trip-table").build();
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg,
new HoodieTableMetaClient(dfs.getConf(), cfg.getBasePath(), true)); new HoodieTableMetaClient(dfs.getConf(), cfg.getBasePath(), true));
boolean result = archiveLog.archiveIfRequired(); boolean result = archiveLog.archiveIfRequired(jsc);
assertTrue(result); assertTrue(result);
} }
@@ -148,7 +166,7 @@ public class TestHoodieCommitArchiveLog {
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg,
new HoodieTableMetaClient(dfs.getConf(), basePath, true)); new HoodieTableMetaClient(dfs.getConf(), basePath, true));
assertTrue(archiveLog.archiveIfRequired()); assertTrue(archiveLog.archiveIfRequired(jsc));
//reload the timeline and remove the remaining commits //reload the timeline and remove the remaining commits
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
@@ -199,7 +217,7 @@ public class TestHoodieCommitArchiveLog {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants()); assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired(); boolean result = archiveLog.archiveIfRequired(jsc);
assertTrue(result); assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants()); assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants());
@@ -222,7 +240,7 @@ public class TestHoodieCommitArchiveLog {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired(); boolean result = archiveLog.archiveIfRequired(jsc);
assertTrue(result); assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100")); assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100"));
@@ -249,7 +267,7 @@ public class TestHoodieCommitArchiveLog {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired(); boolean result = archiveLog.archiveIfRequired(jsc);
assertTrue(result); assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
assertEquals( assertEquals(

View File

@@ -107,7 +107,7 @@ public class TestHoodieCompactor {
public void testCompactionOnCopyOnWriteFail() throws Exception { public void testCompactionOnCopyOnWriteFail() throws Exception {
HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime());
} }
@@ -115,7 +115,7 @@ public class TestHoodieCompactor {
public void testCompactionEmpty() throws Exception { public void testCompactionEmpty() throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = getConfig(); HoodieWriteConfig config = getConfig();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
String newCommitTime = writeClient.startCommit(); String newCommitTime = writeClient.startCommit();
@@ -142,15 +142,15 @@ public class TestHoodieCompactor {
// Update all the 100 records // Update all the 100 records
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
newCommitTime = "101"; newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime); writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records); List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
HoodieIndex index = new HoodieBloomIndex<>(config, jsc); HoodieIndex index = new HoodieBloomIndex<>(config);
updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect(); updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
// Write them to corresponding avro logfiles // Write them to corresponding avro logfiles
HoodieTestUtils HoodieTestUtils
@@ -158,7 +158,7 @@ public class TestHoodieCompactor {
// Verify that all data file has one log file // Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config); table = HoodieTable.getHoodieTable(metaClient, config, jsc);
for (String partitionPath : dataGen.getPartitionPaths()) { for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath)
.collect(Collectors.toList()); .collect(Collectors.toList());
@@ -169,7 +169,7 @@ public class TestHoodieCompactor {
// Do a compaction // Do a compaction
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config); table = HoodieTable.getHoodieTable(metaClient, config, jsc);
JavaRDD<WriteStatus> result = compactor JavaRDD<WriteStatus> result = compactor
.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); .compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime());

View File

@@ -247,7 +247,8 @@ public class TestHoodieMergeHandleDuplicateRecords {
for (int i = 0; i < fullPartitionPaths.length; i++) { for (int i = 0; i < fullPartitionPaths.length; i++) {
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
} }
Dataset<Row> dataSet = HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths); Dataset<Row> dataSet = HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs,
fullPartitionPaths);
return dataSet; return dataSet;
} }

View File

@@ -30,6 +30,7 @@ import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy; import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.Random; import java.util.Random;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.junit.Test; import org.junit.Test;
@@ -127,7 +128,7 @@ public class TestHoodieCompactionStrategy {
Map<Long, List<Long>> sizesMap) { Map<Long, List<Long>> sizesMap) {
List<CompactionOperation> operations = Lists.newArrayList(sizesMap.size()); List<CompactionOperation> operations = Lists.newArrayList(sizesMap.size());
sizesMap.forEach((k, v) -> { sizesMap.forEach((k, v) -> {
operations.add(new CompactionOperation(TestHoodieDataFile.newDataFile(k), operations.add(new CompactionOperation(Optional.of(TestHoodieDataFile.newDataFile(k)),
partitionPaths[new Random().nextInt(partitionPaths.length - 1)], partitionPaths[new Random().nextInt(partitionPaths.length - 1)],
v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), config)); v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), config));
}); });

View File

@@ -77,6 +77,7 @@ public class TestCopyOnWriteTable {
folder.create(); folder.create();
this.basePath = folder.getRoot().getAbsolutePath(); this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
} }
@Test @Test
@@ -90,9 +91,10 @@ public class TestCopyOnWriteTable {
String commitTime = HoodieTestUtils.makeNewCommitTime(); String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig(); HoodieWriteConfig config = makeHoodieClientConfig();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath); HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath,
UUID.randomUUID().toString());
Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName); Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName);
assertTrue(newPath.toString().equals( assertTrue(newPath.toString().equals(
this.basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, unitNumber, fileName))); this.basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, unitNumber, fileName)));
@@ -117,7 +119,7 @@ public class TestCopyOnWriteTable {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
String partitionPath = "/2016/01/31"; String partitionPath = "/2016/01/31";
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Get some records belong to the same partition (2016/01/31) // Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
@@ -187,7 +189,7 @@ public class TestCopyOnWriteTable {
Thread.sleep(1000); Thread.sleep(1000);
String newCommitTime = HoodieTestUtils.makeNewCommitTime(); String newCommitTime = HoodieTestUtils.makeNewCommitTime();
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = new HoodieCopyOnWriteTable(config, metadata); table = new HoodieCopyOnWriteTable(config, jsc);
Iterator<List<WriteStatus>> iter = table Iterator<List<WriteStatus>> iter = table
.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(), .handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(),
updatedRecords.iterator()); updatedRecords.iterator());
@@ -255,7 +257,7 @@ public class TestCopyOnWriteTable {
String firstCommitTime = HoodieTestUtils.makeNewCommitTime(); String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Get some records belong to the same partition (2016/01/31) // Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
@@ -290,7 +292,7 @@ public class TestCopyOnWriteTable {
String commitTime = HoodieTestUtils.makeNewCommitTime(); String commitTime = HoodieTestUtils.makeNewCommitTime();
FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Write a few records, and get atleast one file // Write a few records, and get atleast one file
// 10 records for partition 1, 1 record for partition 2. // 10 records for partition 1, 1 record for partition 2.
@@ -324,7 +326,7 @@ public class TestCopyOnWriteTable {
HoodieWriteConfig config = makeHoodieClientConfig(); HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime(); String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Case 1: // Case 1:
// 10 records for partition 1, 1 record for partition 2. // 10 records for partition 1, 1 record for partition 2.
@@ -372,7 +374,7 @@ public class TestCopyOnWriteTable {
.build()).build(); .build()).build();
String commitTime = HoodieTestUtils.makeNewCommitTime(); String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
List<HoodieRecord> records = new ArrayList<>(); List<HoodieRecord> records = new ArrayList<>();
// Approx 1150 records are written for block size of 64KB // Approx 1150 records are written for block size of 64KB
@@ -411,9 +413,9 @@ public class TestCopyOnWriteTable {
HoodieClientTestUtils.fakeDataFile(basePath, testPartitionPath, "001", "file1", fileSize); HoodieClientTestUtils.fakeDataFile(basePath, testPartitionPath, "001", "file1", fileSize);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath}); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[]{testPartitionPath});
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts); List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
List<HoodieRecord> updateRecords = dataGenerator.generateUpdates("001", numUpdates); List<HoodieRecord> updateRecords = dataGenerator.generateUpdates("001", numUpdates);
for (HoodieRecord updateRec : updateRecords) { for (HoodieRecord updateRec : updateRecords) {

View File

@@ -47,6 +47,7 @@ import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.index.HoodieIndex.IndexType;
import com.uber.hoodie.index.bloom.HoodieBloomIndex; import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
@@ -70,7 +71,6 @@ import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.TemporaryFolder; import org.junit.rules.TemporaryFolder;
@@ -155,7 +155,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent()); assertTrue(deltaCommit.isPresent());
@@ -166,7 +166,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles(); Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent()); assertTrue(!dataFilesToRead.findAny().isPresent());
@@ -209,9 +209,8 @@ public class TestMergeOnReadTable {
assertTrue(dataFilesToRead.findAny().isPresent()); assertTrue(dataFilesToRead.findAny().isPresent());
// verify that there is a commit // verify that there is a commit
HoodieTable table = HoodieTable.getHoodieTable( metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true);
new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), getConfig(false)); HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants();
HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants();
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp(); String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
assertTrue(HoodieTimeline.compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER)); assertTrue(HoodieTimeline.compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER));
@@ -263,7 +262,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent()); assertTrue(deltaCommit.isPresent());
@@ -274,7 +273,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles(); Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent()); assertTrue(!dataFilesToRead.findAny().isPresent());
@@ -372,7 +371,7 @@ public class TestMergeOnReadTable {
client.rollback(newCommitTime); client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCompletedCommitTimeline(), allFiles); hoodieTable.getCompletedCommitTimeline(), allFiles);
@@ -408,7 +407,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent()); assertTrue(deltaCommit.isPresent());
@@ -419,7 +418,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles(); Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
assertTrue(!dataFilesToRead.findAny().isPresent()); assertTrue(!dataFilesToRead.findAny().isPresent());
@@ -456,7 +455,7 @@ public class TestMergeOnReadTable {
client.rollback(newCommitTime); client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles);
dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
@@ -482,8 +481,8 @@ public class TestMergeOnReadTable {
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get() final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get()
.getTimestamp(); .getTimestamp();
@@ -500,8 +499,8 @@ public class TestMergeOnReadTable {
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
assertFalse(roView.getLatestDataFiles().filter(file -> { assertFalse(roView.getLatestDataFiles().filter(file -> {
if (compactedCommitTime.equals(file.getCommitTime())) { if (compactedCommitTime.equals(file.getCommitTime())) {
@@ -531,7 +530,7 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); Optional<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent()); assertTrue(deltaCommit.isPresent());
@@ -542,7 +541,7 @@ public class TestMergeOnReadTable {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
hoodieTable.getCommitsTimeline().filterCompletedInstants(), allFiles); metaClient.getCommitsTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles(); Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
Map<String, Long> parquetFileIdToSize = dataFilesToRead.collect( Map<String, Long> parquetFileIdToSize = dataFilesToRead.collect(
Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize));
@@ -591,7 +590,6 @@ public class TestMergeOnReadTable {
} }
@Test @Test
@Ignore
public void testLogFileCountsAfterCompaction() throws Exception { public void testLogFileCountsAfterCompaction() throws Exception {
// insert 100 records // insert 100 records
HoodieWriteConfig config = getConfig(true); HoodieWriteConfig config = getConfig(true);
@@ -606,15 +604,15 @@ public class TestMergeOnReadTable {
// Update all the 100 records // Update all the 100 records
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
newCommitTime = "101"; newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime); writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records); List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
HoodieIndex index = new HoodieBloomIndex<>(config, jsc); HoodieIndex index = new HoodieBloomIndex<>(config);
updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect(); updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
// Write them to corresponding avro logfiles // Write them to corresponding avro logfiles
HoodieTestUtils HoodieTestUtils
@@ -623,7 +621,7 @@ public class TestMergeOnReadTable {
// Verify that all data file has one log file // Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config); table = HoodieTable.getHoodieTable(metaClient, config, jsc);
for (String partitionPath : dataGen.getPartitionPaths()) { for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath)
.collect(Collectors.toList()); .collect(Collectors.toList());
@@ -634,14 +632,14 @@ public class TestMergeOnReadTable {
// Do a compaction // Do a compaction
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config); table = HoodieTable.getHoodieTable(metaClient, config, jsc);
String commitTime = writeClient.startCompaction(); String commitTime = writeClient.startCompaction();
JavaRDD<WriteStatus> result = writeClient.compact(commitTime); JavaRDD<WriteStatus> result = writeClient.compact(commitTime);
// Verify that recently written compacted data file has no log file // Verify that recently written compacted data file has no log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config); table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
assertTrue("Compaction commit should be > than last insert", HoodieTimeline.compareTimestamps( assertTrue("Compaction commit should be > than last insert", HoodieTimeline.compareTimestamps(
@@ -677,7 +675,7 @@ public class TestMergeOnReadTable {
// total time taken for creating files should be greater than 0 // total time taken for creating files should be greater than 0
long totalCreateTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalCreateTime()) long totalCreateTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalCreateTime())
.reduce((a,b) -> a + b).intValue(); .reduce((a, b) -> a + b).intValue();
Assert.assertTrue(totalCreateTime > 0); Assert.assertTrue(totalCreateTime > 0);
// Update all the 100 records // Update all the 100 records
@@ -690,7 +688,7 @@ public class TestMergeOnReadTable {
writeClient.commit(newCommitTime, statuses); writeClient.commit(newCommitTime, statuses);
// total time taken for upsert all records should be greater than 0 // total time taken for upsert all records should be greater than 0
long totalUpsertTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalUpsertTime()) long totalUpsertTime = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalUpsertTime())
.reduce((a,b) -> a + b).intValue(); .reduce((a, b) -> a + b).intValue();
Assert.assertTrue(totalUpsertTime > 0); Assert.assertTrue(totalUpsertTime > 0);
// Do a compaction // Do a compaction
@@ -699,22 +697,168 @@ public class TestMergeOnReadTable {
writeClient.commitCompaction(commitTime, statuses); writeClient.commitCompaction(commitTime, statuses);
// total time taken for scanning log files should be greater than 0 // total time taken for scanning log files should be greater than 0
long timeTakenForScanner = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalScanTime()) long timeTakenForScanner = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalScanTime())
.reduce((a,b) -> a + b).longValue(); .reduce((a, b) -> a + b).longValue();
Assert.assertTrue(timeTakenForScanner > 0); Assert.assertTrue(timeTakenForScanner > 0);
} }
@Test
public void testSimpleInsertsGeneratedIntoLogFiles() throws Exception {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime);
writeClient.commit(newCommitTime, statuses);
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config,
jsc);
TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView();
long numLogFiles = 0;
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() > 0);
numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count();
}
Assert.assertTrue(numLogFiles > 0);
// Do a compaction
String commitTime = writeClient.startCompaction();
statuses = writeClient.compact(commitTime);
Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles);
Assert.assertEquals(statuses.count(), numLogFiles);
writeClient.commitCompaction(commitTime, statuses);
}
@Test
public void testInsertsGeneratedIntoLogFilesRollback() throws Exception {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime);
// trigger an action
List<WriteStatus> writeStatuses = statuses.collect();
// Ensure that inserts are written to only log files
Assert.assertEquals(writeStatuses.stream().filter(writeStatus -> !writeStatus.getStat().getPath().contains("log")
).count(), 0);
Assert.assertTrue(writeStatuses.stream().filter(writeStatus -> writeStatus.getStat().getPath().contains("log")
).count() > 0);
// rollback a failed commit
boolean rollback = writeClient.rollback(newCommitTime);
Assert.assertTrue(rollback);
newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime);
// insert 100 records
records = dataGen.generateInserts(newCommitTime, 100);
recordsRDD = jsc.parallelize(records, 1);
statuses = writeClient.insert(recordsRDD, newCommitTime);
writeClient.commit(newCommitTime, statuses);
// rollback a successful commit
writeClient.rollback(newCommitTime);
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView();
long numLogFiles = 0;
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() == 0);
numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count();
}
Assert.assertTrue(numLogFiles == 0);
}
@Test
public void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction() throws Exception {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime);
writeClient.commit(newCommitTime, statuses);
// trigger an action
statuses.collect();
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config,
jsc);
TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView();
long numLogFiles = 0;
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() > 0);
numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count();
}
Assert.assertTrue(numLogFiles > 0);
// Do a compaction
newCommitTime = writeClient.startCompaction();
statuses = writeClient.compact(newCommitTime);
// Ensure all log files have been compacted into parquet files
Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles);
Assert.assertEquals(statuses.count(), numLogFiles);
writeClient.commitCompaction(newCommitTime, statuses);
// Trigger a rollback of compaction
writeClient.rollback(newCommitTime);
for (String partitionPath : dataGen.getPartitionPaths()) {
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getDataFile().isPresent()).count() == 0);
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
fileSlice.getLogFiles().count() > 0).count() > 0);
}
}
private HoodieWriteConfig getConfig(Boolean autoCommit) { private HoodieWriteConfig getConfig(Boolean autoCommit) {
return getConfigBuilder(autoCommit).build(); return getConfigBuilder(autoCommit).build();
} }
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) { private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
return getConfigBuilder(autoCommit, IndexType.BLOOM);
}
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig( .withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false) HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
.withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
.forTable("test-trip-table") .forTable("test-trip-table")
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build());
} }
private void assertNoWriteErrors(List<WriteStatus> statuses) { private void assertNoWriteErrors(List<WriteStatus> statuses) {

View File

@@ -16,12 +16,15 @@
package com.uber.hoodie.common.table; package com.uber.hoodie.common.table;
import static com.uber.hoodie.common.model.HoodieTableType.MERGE_ON_READ;
import com.uber.hoodie.common.SerializableConfiguration; import com.uber.hoodie.common.SerializableConfiguration;
import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline; import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.DatasetNotFoundException;
import com.uber.hoodie.exception.HoodieException;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
@@ -198,7 +201,7 @@ public class HoodieTableMetaClient implements Serializable {
Properties properties = new Properties(); Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name()); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
if (tableType == HoodieTableType.MERGE_ON_READ) { if (tableType == MERGE_ON_READ) {
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName); properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
} }
return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties); return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties);
@@ -254,6 +257,53 @@ public class HoodieTableMetaClient implements Serializable {
return fs.listStatus(metaPath, nameFilter); return fs.listStatus(metaPath, nameFilter);
} }
/**
* Get the commit timeline visible for this table
*/
public HoodieTimeline getCommitsTimeline() {
switch (this.getTableType()) {
case COPY_ON_WRITE:
return getActiveTimeline().getCommitTimeline();
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits
// Include commit action to be able to start doing a MOR over a COW dataset - no
// migration required
return getActiveTimeline().getCommitsTimeline();
default:
throw new HoodieException("Unsupported table type :" + this.getTableType());
}
}
/**
* Get the compacted commit timeline visible for this table
*/
public HoodieTimeline getCommitTimeline() {
switch (this.getTableType()) {
case COPY_ON_WRITE:
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits in tagging
return getActiveTimeline().getCommitTimeline();
default:
throw new HoodieException("Unsupported table type :" + this.getTableType());
}
}
/**
* Gets the commit action type
*/
public String getCommitActionType() {
switch (this.getTableType()) {
case COPY_ON_WRITE:
return HoodieActiveTimeline.COMMIT_ACTION;
case MERGE_ON_READ:
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
default:
throw new HoodieException(
"Could not commit on unknown storage type " + this.getTableType());
}
}
@Override @Override
public boolean equals(Object o) { public boolean equals(Object o) {
if (this == o) { if (this == o) {

View File

@@ -91,13 +91,12 @@ public abstract class AbstractHoodieLogRecordScanner {
private AtomicLong totalCorruptBlocks = new AtomicLong(0); private AtomicLong totalCorruptBlocks = new AtomicLong(0);
// Store the last instant log blocks (needed to implement rollback) // Store the last instant log blocks (needed to implement rollback)
private Deque<HoodieLogBlock> currentInstantLogBlocks = new ArrayDeque<>(); private Deque<HoodieLogBlock> currentInstantLogBlocks = new ArrayDeque<>();
// Progress // Progress
private float progress = 0.0f; private float progress = 0.0f;
public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, // TODO (NA) - Change this to a builder, this constructor is too long
Schema readerSchema, String latestInstantTime, public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, Schema readerSchema,
boolean readBlocksLazily, boolean reverseReader, int bufferSize) { String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize) {
this.readerSchema = readerSchema; this.readerSchema = readerSchema;
this.latestInstantTime = latestInstantTime; this.latestInstantTime = latestInstantTime;
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath); this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath);

View File

@@ -247,6 +247,13 @@ public class FSUtils {
return LOG_FILE_PREFIX + String.format("%s_%s%s*", fileId, commitTime, logFileExtension); return LOG_FILE_PREFIX + String.format("%s_%s%s*", fileId, commitTime, logFileExtension);
} }
public static boolean isLogFile(Path logPath) {
Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName());
if (!matcher.find()) {
return false;
}
return true;
}
/** /**
* Get the latest log file written from the list of log files passed in * Get the latest log file written from the list of log files passed in

View File

@@ -24,7 +24,6 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.table.HoodieTable;
import java.util.List; import java.util.List;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@@ -66,15 +65,13 @@ public class HoodieDataSourceHelpers {
* could be fed into the datasource options. * could be fed into the datasource options.
*/ */
public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) { public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) {
HoodieTable table = HoodieTable HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), if (metaClient.getTableType().equals(HoodieTableType.MERGE_ON_READ)) {
null); return metaClient.getActiveTimeline().getTimelineOfActions(
if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) {
return table.getActiveTimeline().getTimelineOfActions(
Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
HoodieActiveTimeline.DELTA_COMMIT_ACTION)); HoodieActiveTimeline.DELTA_COMMIT_ACTION));
} else { } else {
return table.getCommitTimeline().filterCompletedInstants(); return metaClient.getCommitTimeline().filterCompletedInstants();
} }
} }
} }

View File

@@ -21,6 +21,7 @@ package com.uber.hoodie
import com.uber.hoodie.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieTableType} import com.uber.hoodie.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieTableType}
import com.uber.hoodie.common.table.HoodieTableMetaClient import com.uber.hoodie.common.table.HoodieTableMetaClient
import com.uber.hoodie.common.util.ParquetUtils import com.uber.hoodie.common.util.ParquetUtils
import com.uber.hoodie.config.HoodieWriteConfig
import com.uber.hoodie.exception.HoodieException import com.uber.hoodie.exception.HoodieException
import com.uber.hoodie.table.HoodieTable import com.uber.hoodie.table.HoodieTable
import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.Path
@@ -52,8 +53,10 @@ class IncrementalRelation(val sqlContext: SQLContext,
if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) { if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) {
throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets") throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets")
} }
val hoodieTable = HoodieTable.getHoodieTable(metaClient, null) // TODO : Figure out a valid HoodieWriteConfig
val commitTimeline = hoodieTable.getCommitTimeline.filterCompletedInstants(); val hoodieTable = HoodieTable.getHoodieTable(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(),
sqlContext.sparkContext)
val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants();
if (commitTimeline.empty()) { if (commitTimeline.empty()) {
throw new HoodieException("No instants to incrementally pull") throw new HoodieException("No instants to incrementally pull")
} }