From 113c2d894f00fe057e0536792c510fb1460deb10 Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Sun, 13 May 2018 16:25:11 -0700 Subject: [PATCH 1/2] Adding ability for inserts to be written to log files --- .../com/uber/hoodie/HoodieReadClient.java | 15 +- .../com/uber/hoodie/HoodieWriteClient.java | 82 +++---- .../java/com/uber/hoodie/WriteStatus.java | 12 +- .../hoodie/config/HoodieStorageConfig.java | 10 + .../uber/hoodie/config/HoodieWriteConfig.java | 6 +- .../hoodie/func/BulkInsertMapFunction.java | 2 +- ...ava => CopyOnWriteLazyInsertIterable.java} | 33 +-- .../func/MergeOnReadLazyInsertIterable.java | 73 +++++++ .../com/uber/hoodie/index/HoodieIndex.java | 15 +- .../uber/hoodie/index/InMemoryHashIndex.java | 12 +- .../hoodie/index/bloom/HoodieBloomIndex.java | 51 ++--- .../bloom/HoodieBloomIndexCheckFunction.java | 16 +- .../hoodie/index/bucketed/BucketedIndex.java | 14 +- .../uber/hoodie/index/hbase/HBaseIndex.java | 27 ++- .../uber/hoodie/io/HoodieAppendHandle.java | 193 +++++++++++------ .../com/uber/hoodie/io/HoodieCleanHelper.java | 2 +- .../hoodie/io/HoodieCommitArchiveLog.java | 9 +- .../uber/hoodie/io/HoodieCreateHandle.java | 41 +++- .../com/uber/hoodie/io/HoodieIOHandle.java | 29 +++ .../com/uber/hoodie/io/HoodieMergeHandle.java | 5 +- .../io/compact/CompactionOperation.java | 34 +-- .../compact/HoodieRealtimeTableCompactor.java | 29 ++- .../compact/strategy/CompactionStrategy.java | 11 +- .../LogFileSizeBasedCompactionStrategy.java | 5 +- .../io/storage/HoodieParquetWriter.java | 2 +- .../storage/HoodieStorageWriterFactory.java | 11 +- .../io/storage/log/HoodieLogWriter.java | 117 ++++++++++ .../hoodie/table/HoodieCopyOnWriteTable.java | 55 +++-- .../hoodie/table/HoodieMergeOnReadTable.java | 71 +++++-- .../com/uber/hoodie/table/HoodieTable.java | 65 ++---- .../java/com/uber/hoodie/TestCleaner.java | 62 +++--- .../com/uber/hoodie/TestClientRollback.java | 6 +- .../com/uber/hoodie/TestHoodieClientBase.java | 6 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 10 +- .../hoodie/common/HoodieClientTestUtils.java | 13 +- .../common/HoodieTestDataGenerator.java | 4 +- .../func/TestBoundedInMemoryExecutor.java | 6 +- .../hoodie/func/TestBoundedInMemoryQueue.java | 2 +- .../hoodie/func/TestUpdateMapFunction.java | 21 +- .../com/uber/hoodie/index/TestHbaseIndex.java | 40 ++-- .../uber/hoodie/index/TestHoodieIndex.java | 43 +++- .../index/bloom/TestHoodieBloomIndex.java | 49 +++-- .../hoodie/io/TestHoodieCommitArchiveLog.java | 32 ++- .../uber/hoodie/io/TestHoodieCompactor.java | 12 +- ...TestHoodieMergeHandleDuplicateRecords.java | 3 +- .../TestHoodieCompactionStrategy.java | 3 +- .../hoodie/table/TestCopyOnWriteTable.java | 22 +- .../hoodie/table/TestMergeOnReadTable.java | 200 +++++++++++++++--- .../uber/hoodie/common/model/FileSlice.java | 40 +++- .../hoodie/common/model/HoodieFileGroup.java | 167 +++++++++++++-- .../hoodie/common/model/HoodieLogFile.java | 14 +- .../common/table/HoodieTableMetaClient.java | 52 ++++- .../log/AbstractHoodieLogRecordScanner.java | 7 +- .../table/view/HoodieTableFileSystemView.java | 13 +- .../com/uber/hoodie/common/util/FSUtils.java | 7 + .../uber/hoodie/HoodieDataSourceHelpers.java | 11 +- .../com/uber/hoodie/IncrementalRelation.scala | 7 +- 57 files changed, 1369 insertions(+), 530 deletions(-) rename hoodie-client/src/main/java/com/uber/hoodie/func/{LazyInsertIterable.java => CopyOnWriteLazyInsertIterable.java} (86%) create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java create mode 100644 hoodie-client/src/main/java/com/uber/hoodie/io/storage/log/HoodieLogWriter.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java index 9a2bed3abfe1..2d8b59208a53 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -64,6 +64,7 @@ public class HoodieReadClient implements Serializ private final HoodieTimeline commitTimeline; private HoodieTable hoodieTable; private transient Optional sqlContextOpt; + private String basePath; /** * @param basePath path to Hoodie dataset @@ -94,10 +95,12 @@ public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) { this.jsc = jsc; this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); // Create a Hoodie table which encapsulated the commits and files visible + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true); this.hoodieTable = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), - clientConfig); - this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants(); + .getHoodieTable(metaClient, + clientConfig, jsc); + this.basePath = basePath; + this.commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants(); this.index = HoodieIndex.createIndex(clientConfig, jsc); this.sqlContextOpt = Optional.absent(); } @@ -128,7 +131,7 @@ public Dataset read(JavaRDD hoodieKeys, int parallelism) throws assertSqlContext(); JavaPairRDD> keyToFileRDD = index - .fetchRecordLocation(hoodieKeys, hoodieTable); + .fetchRecordLocation(hoodieKeys, jsc, hoodieTable.getMetaClient()); List paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent()) .map(keyFileTuple -> keyFileTuple._2().get()).collect(); @@ -156,7 +159,7 @@ public Dataset read(JavaRDD hoodieKeys, int parallelism) throws * file */ public JavaPairRDD> checkExists(JavaRDD hoodieKeys) { - return index.fetchRecordLocation(hoodieKeys, hoodieTable); + return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable.getMetaClient()); } /** @@ -180,6 +183,6 @@ public JavaRDD> filterExists(JavaRDD> hoodieReco */ public JavaRDD> tagLocation(JavaRDD> hoodieRecords) throws HoodieIndexException { - return index.tagLocation(hoodieRecords, hoodieTable); + return index.tagLocation(hoodieRecords, jsc, this.hoodieTable.getMetaClient()); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 16aba54513ab..5b506d18125c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -130,7 +130,7 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, public static SparkConf registerClasses(SparkConf conf) { conf.registerKryoClasses( - new Class[] {HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); + new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); return conf; } @@ -144,9 +144,9 @@ public static SparkConf registerClasses(SparkConf conf) { public JavaRDD> filterExists(JavaRDD> hoodieRecords) { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); - JavaRDD> recordsWithLocation = index.tagLocation(hoodieRecords, table); + JavaRDD> recordsWithLocation = index.tagLocation(hoodieRecords, jsc, table.getMetaClient()); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); } @@ -161,7 +161,7 @@ public JavaRDD upsert(JavaRDD> records, final Strin config.shouldCombineBeforeUpsert(), records, config.getUpsertShuffleParallelism()); // perform index loop up to get existing location of records - JavaRDD> taggedRecords = index.tagLocation(dedupedRecords, table); + JavaRDD> taggedRecords = index.tagLocation(dedupedRecords, jsc, table.getMetaClient()); return upsertRecordsInternal(taggedRecords, commitTime, table, true); } catch (Throwable e) { if (e instanceof HoodieUpsertException) { @@ -178,7 +178,7 @@ public JavaRDD upsert(JavaRDD> records, final Strin * needed. * * @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 */ public JavaRDD upsertPreppedRecords(JavaRDD> preppedRecords, @@ -202,7 +202,7 @@ public JavaRDD upsertPreppedRecords(JavaRDD> preppe * 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 * - * @param records HoodieRecords to insert + * @param records HoodieRecords to insert * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ @@ -230,7 +230,7 @@ public JavaRDD insert(JavaRDD> records, final Strin * The prepared HoodieRecords should be de-duped if needed. * * @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 */ public JavaRDD insertPreppedRecords(JavaRDD> preppedRecords, @@ -256,7 +256,7 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe * attempts to control the numbers of files with less memory compared to the {@link * HoodieWriteClient#insert(JavaRDD, String)} * - * @param records HoodieRecords to insert + * @param records HoodieRecords to insert * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ @@ -276,10 +276,10 @@ public JavaRDD bulkInsert(JavaRDD> records, * partitioner. If specified then it will be used for repartitioning records. See {@link * UserDefinedBulkInsertPartitioner}. * - * @param records HoodieRecords to insert - * @param commitTime Commit Time handle + * @param records HoodieRecords to insert + * @param commitTime Commit Time handle * @param bulkInsertPartitioner If specified then it will be used to partition input records - * before they are inserted into hoodie. + * before they are inserted into hoodie. * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD bulkInsert(JavaRDD> records, final String commitTime, @@ -310,10 +310,10 @@ public JavaRDD bulkInsert(JavaRDD> records, final S * partitioner. If specified then it will be used for repartitioning records. See {@link * UserDefinedBulkInsertPartitioner}. * - * @param preppedRecords HoodieRecords to insert - * @param commitTime Commit Time handle + * @param preppedRecords HoodieRecords to insert + * @param commitTime Commit Time handle * @param bulkInsertPartitioner If specified then it will be used to partition input records - * before they are inserted into hoodie. + * before they are inserted into hoodie. * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, @@ -390,6 +390,7 @@ private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, Hood partitionStat.getUpdateLocationToCount().entrySet().stream().forEach(entry -> { HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setFileId(entry.getKey()); + // TODO : Write baseCommitTime is possible here ? writeStat.setPrevCommit(entry.getValue().getKey()); writeStat.setNumUpdateWrites(entry.getValue().getValue()); metadata.addWriteStat(path.toString(), writeStat); @@ -450,10 +451,12 @@ private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, Workload private JavaRDD updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieTable table, String commitTime) { // Update the index back - JavaRDD statuses = index.updateLocation(writeStatusRDD, table); + JavaRDD statuses = index.updateLocation(writeStatusRDD, jsc, table.getMetaClient()); // Trigger the insert and collect statuses statuses = statuses.persist(config.getWriteStatusStorageLevel()); - commitOnAutoCommit(commitTime, statuses, table.getCommitActionType()); + commitOnAutoCommit(commitTime, statuses, + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true) + .getCommitActionType()); return statuses; } @@ -476,9 +479,8 @@ public boolean commit(String commitTime, JavaRDD writeStatuses) { */ public boolean commit(String commitTime, JavaRDD writeStatuses, Optional> extraMetadata) { - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - return commit(commitTime, writeStatuses, extraMetadata, table.getCommitActionType()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); + return commit(commitTime, writeStatuses, extraMetadata, metaClient.getCommitActionType()); } private boolean commit(String commitTime, JavaRDD writeStatuses, @@ -487,7 +489,7 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, logger.info("Commiting " + commitTime); // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); @@ -531,7 +533,7 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, // We cannot have unbounded commit files. Archive commits if we have to archive HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config, new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)); - archiveLog.archiveIfRequired(); + archiveLog.archiveIfRequired(jsc); if (config.isAutoClean()) { // Call clean to cleanup if there is anything to cleanup after the commit, logger.info("Auto cleaning is enabled. Running cleaner now"); @@ -568,13 +570,13 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, *

* 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 * @return true if the savepoint was created successfully */ public boolean savepoint(String user, String comment) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); if (table.getCompletedCommitTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } @@ -595,13 +597,13 @@ public boolean savepoint(String user, String comment) { * Savepoint should be on a commit that could not have been cleaned. * * @param commitTime - commit that should be savepointed - * @param user - User creating the savepoint - * @param comment - Comment for the savepoint + * @param user - User creating the savepoint + * @param comment - Comment for the savepoint * @return true if the savepoint was created successfully */ public boolean savepoint(String commitTime, String user, String comment) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); Optional cleanInstant = table.getCompletedCleanTimeline().lastInstant(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, @@ -662,7 +664,7 @@ public boolean savepoint(String commitTime, String user, String comment) { */ public void deleteSavepoint(String savepointTime) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, @@ -688,9 +690,9 @@ public void deleteSavepoint(String savepointTime) { */ public boolean rollbackToSavepoint(String savepointTime) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieTimeline commitTimeline = table.getCommitsTimeline(); + HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline(); HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); @@ -737,7 +739,7 @@ private void rollback(List commits) { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieTimeline inflightTimeline = table.getInflightCommitTimeline(); HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); @@ -841,7 +843,7 @@ private void clean(String startCleanTime) throws HoodieIOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); List cleanStats = table.clean(jsc); if (cleanStats.isEmpty()) { @@ -890,9 +892,9 @@ public String startCommit() { public void startCommitWithTime(String commitTime) { logger.info("Generate a new commit time " + commitTime); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - String commitActionType = table.getCommitActionType(); + String commitActionType = table.getMetaClient().getCommitActionType(); activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); } @@ -912,7 +914,7 @@ public String startCompaction() { */ public void startCompactionWithTime(String commitTime) { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); String commitActionType = HoodieTimeline.COMMIT_ACTION; activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); @@ -925,7 +927,7 @@ public void startCompactionWithTime(String commitTime) { public JavaRDD compact(String commitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable 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 writeContext = metrics.getCommitCtx(); JavaRDD statuses = table.compact(jsc, commitTime); @@ -961,7 +963,7 @@ private void forceCompact(String compactionCommitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); // TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction and // then use getTableAndInitCtx Timer.Context writeContext = metrics.getCommitCtx(); @@ -1048,8 +1050,8 @@ JavaRDD> deduplicateRecords(JavaRDD> records, */ private void rollbackInflightCommits() { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); + HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterInflights(); List commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); Collections.reverse(commits); @@ -1061,8 +1063,8 @@ private void rollbackInflightCommits() { private HoodieTable getTableAndInitCtx() { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); - if (table.getCommitActionType() == HoodieTimeline.COMMIT_ACTION) { + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); + if (table.getMetaClient().getCommitActionType() == HoodieTimeline.COMMIT_ACTION) { writeContext = metrics.getCommitCtx(); } else { writeContext = metrics.getDeltaCommitCtx(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java b/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java index c28db2567b8c..14a01775386d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java @@ -53,10 +53,10 @@ public class WriteStatus implements Serializable { * aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus * objects are collected in Spark Driver. * - * @param record deflated {@code HoodieRecord} containing information that uniquely identifies - * it. + * @param record deflated {@code HoodieRecord} containing information that uniquely identifies + * it. * @param optionalRecordMetadata optional metadata related to data contained in {@link - * HoodieRecord} before deflation. + * HoodieRecord} before deflation. */ public void markSuccess(HoodieRecord record, Optional> optionalRecordMetadata) { @@ -69,10 +69,10 @@ public void markSuccess(HoodieRecord record, * aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus * objects are collected in Spark Driver. * - * @param record deflated {@code HoodieRecord} containing information that uniquely identifies - * it. + * @param record deflated {@code HoodieRecord} containing information that uniquely identifies + * it. * @param optionalRecordMetadata optional metadata related to data contained in {@link - * HoodieRecord} before deflation. + * HoodieRecord} before deflation. */ public void markFailure(HoodieRecord record, Throwable t, Optional> optionalRecordMetadata) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java index b72e6f743453..0403b62ebb46 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieStorageConfig.java @@ -44,6 +44,9 @@ public class HoodieStorageConfig extends DefaultHoodieConfig { public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio"; // Default compression ratio for parquet 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) { super(props); @@ -102,6 +105,11 @@ public Builder parquetCompressionRatio(double parquetCompressionRatio) { return this; } + public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressionRatio) { + props.setProperty(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio)); + return this; + } + public HoodieStorageConfig build() { HoodieStorageConfig config = new HoodieStorageConfig(props); setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES), @@ -116,6 +124,8 @@ public HoodieStorageConfig build() { LOGFILE_SIZE_MAX_BYTES, DEFAULT_LOGFILE_SIZE_MAX_BYTES); setDefaultOnCondition(props, !props.containsKey(PARQUET_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; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java index e0fe99379e0d..7a417225ab98 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java @@ -322,6 +322,10 @@ public double getParquetCompressionRatio() { 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 **/ @@ -345,7 +349,7 @@ public int getGraphiteServerPort() { public String getGraphiteMetricPrefix() { return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX); } - + /** * memory configs */ diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java index c8a39b6857cd..e530e86bd04d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/BulkInsertMapFunction.java @@ -46,6 +46,6 @@ public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config, @Override public Iterator> call(Integer partition, Iterator> sortedRecordItr) throws Exception { - return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable); + return new CopyOnWriteLazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java similarity index 86% rename from hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java rename to hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java index 6fdf41c3c514..d08ac4b6342b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java @@ -33,6 +33,7 @@ import java.util.List; import java.util.Optional; import java.util.Set; +import java.util.UUID; import java.util.function.Function; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -43,15 +44,15 @@ * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new * files. */ -public class LazyInsertIterable extends +public class CopyOnWriteLazyInsertIterable extends LazyIterableIterator, List> { - private final HoodieWriteConfig hoodieConfig; - private final String commitTime; - private final HoodieTable hoodieTable; - private Set partitionsCleaned; + protected final HoodieWriteConfig hoodieConfig; + protected final String commitTime; + protected final HoodieTable hoodieTable; + protected Set partitionsCleaned; - public LazyInsertIterable(Iterator> sortedRecordItr, HoodieWriteConfig config, + public CopyOnWriteLazyInsertIterable(Iterator> sortedRecordItr, HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { super(sortedRecordItr); this.partitionsCleaned = new HashSet<>(); @@ -89,7 +90,7 @@ protected List computeNext() { final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig); bufferedIteratorExecutor = new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, - new InsertHandler(), getTransformFunction(schema)); + getInsertHandler(), getTransformFunction(schema)); final List result = bufferedIteratorExecutor.execute(); assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); return result; @@ -107,15 +108,19 @@ protected void end() { } + protected InsertHandler getInsertHandler() { + return new InsertHandler(); + } + /** * Consumes stream of hoodie records from in-memory queue and * writes to one or more create-handles */ - private class InsertHandler extends + protected class InsertHandler extends BoundedInMemoryQueueConsumer, Optional>, List> { - private final List statuses = new ArrayList<>(); - private HoodieCreateHandle handle; + protected final List statuses = new ArrayList<>(); + protected HoodieIOHandle handle; @Override protected void consumeOneRecord(Tuple2, Optional> payload) { @@ -132,7 +137,8 @@ protected void consumeOneRecord(Tuple2, Optional> // lazily initialize the handle, for the first time 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())) { @@ -142,7 +148,8 @@ protected void consumeOneRecord(Tuple2, Optional> // handle is full. statuses.add(handle.close()); // Need to handle the rejected payload & open new handle - handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath()); + handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), UUID + .randomUUID().toString()); handle.write(insertPayload, payload._2()); // we should be able to write 1 payload. } } @@ -161,4 +168,4 @@ protected List getResult() { return statuses; } } -} +} \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java new file mode 100644 index 000000000000..110cc09d1e22 --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/MergeOnReadLazyInsertIterable.java @@ -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 extends + CopyOnWriteLazyInsertIterable { + + public MergeOnReadLazyInsertIterable(Iterator> sortedRecordItr, HoodieWriteConfig config, + String commitTime, HoodieTable hoodieTable) { + super(sortedRecordItr, config, commitTime, hoodieTable); + } + + @Override + protected InsertHandler getInsertHandler() { + return new MergeOnReadInsertHandler(); + } + + protected class MergeOnReadInsertHandler extends InsertHandler { + + @Override + protected void consumeOneRecord(Tuple2, Optional> payload) { + final HoodieRecord insertPayload = payload._1(); + List 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. + } + } + } + +} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java index 5733237b44c7..cf66d98c1ace 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java @@ -21,12 +21,12 @@ import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.index.bloom.HoodieBloomIndex; import com.uber.hoodie.index.bucketed.BucketedIndex; import com.uber.hoodie.index.hbase.HBaseIndex; -import com.uber.hoodie.table.HoodieTable; import java.io.Serializable; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -38,13 +38,12 @@ public abstract class HoodieIndex implements Serializable { protected final HoodieWriteConfig config; - protected transient JavaSparkContext jsc = null; protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) { this.config = config; - this.jsc = jsc; } + public static HoodieIndex createIndex(HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException { switch (config.getIndexType()) { @@ -68,22 +67,24 @@ public static HoodieIndex createIndex(HoodieW * file */ public abstract JavaPairRDD> fetchRecordLocation( - JavaRDD hoodieKeys, final HoodieTable table); + JavaRDD hoodieKeys, final JavaSparkContext jsc, + HoodieTableMetaClient metaClient); /** * Looks up the index and tags each incoming record with a location of a file that contains the * row (if it is actually present) */ public abstract JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieTable hoodieTable) throws HoodieIndexException; + JavaSparkContext jsc, HoodieTableMetaClient metaClient) throws HoodieIndexException; /** * Extracts the location of written records, and updates the index. *

* TODO(vc): We may need to propagate the record as well in a WriteStatus class */ - public abstract JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieTable hoodieTable) throws HoodieIndexException; + public abstract JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, + HoodieTableMetaClient metaClient) + throws HoodieIndexException; /** * Rollback the efffects of the commit made at commitTime. diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java index 6338e60bb1a1..d6792886065d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java @@ -22,8 +22,8 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.table.HoodieTable; import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -50,19 +50,19 @@ public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) { @Override public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, - final HoodieTable table) { + JavaSparkContext jsc, HoodieTableMetaClient metaClient) { throw new UnsupportedOperationException("InMemory index does not implement check exist yet"); } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieTable hoodieTable) { + public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, + HoodieTableMetaClient metaClient) { return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true); } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieTable hoodieTable) { + public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, + HoodieTableMetaClient metaClient) { return writeStatusRDD.map(new Function() { @Override public WriteStatus call(WriteStatus writeStatus) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index b85f0fed979f..1b9fcf00322e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -30,13 +30,15 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.ParquetUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.MetadataNotFoundException; import com.uber.hoodie.index.HoodieIndex; -import com.uber.hoodie.table.HoodieTable; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -69,8 +71,8 @@ public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) { } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, - final HoodieTable hoodieTable) { + public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, + HoodieTableMetaClient metaClient) { // Step 0: cache the input record RDD if (config.getBloomIndexUseCaching()) { @@ -82,8 +84,7 @@ public JavaRDD> tagLocation(JavaRDD> recordRDD, .mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())); // Lookup indexes for all the partition/recordkey pair - JavaPairRDD rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, - hoodieTable); + JavaPairRDD rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, metaClient); // Cache the result, for subsequent stages. if (config.getBloomIndexUseCaching()) { @@ -108,13 +109,12 @@ public JavaRDD> tagLocation(JavaRDD> recordRDD, } public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, - final HoodieTable table) { + JavaSparkContext jsc, HoodieTableMetaClient metaClient) { JavaPairRDD partitionRecordKeyPairRDD = hoodieKeys .mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey())); // Lookup indexes for all the partition/recordkey pair - JavaPairRDD rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, - table); + JavaPairRDD rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, metaClient); JavaPairRDD rowKeyHoodieKeyPairRDD = hoodieKeys .mapToPair(key -> new Tuple2<>(key.getRecordKey(), key)); @@ -125,7 +125,7 @@ public JavaPairRDD> fetchRecordLocation(JavaRDD> fetchRecordLocation(JavaRDD lookupIndex( - JavaPairRDD partitionRecordKeyPairRDD, final HoodieTable hoodieTable) { + JavaPairRDD partitionRecordKeyPairRDD, final JavaSparkContext + jsc, final HoodieTableMetaClient metaClient) { // Obtain records per partition, in the incoming records Map recordsPerPartition = partitionRecordKeyPairRDD.countByKey(); List affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet()); // Step 2: Load all involved files as pairs - List> fileInfoList = loadInvolvedFiles( - affectedPartitionPathList, hoodieTable); + List> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, jsc, + metaClient); final Map> partitionToFileInfo = fileInfoList.stream() .collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList()))); @@ -154,8 +155,8 @@ private JavaPairRDD lookupIndex( // that contains it. int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD); - return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo, - partitionRecordKeyPairRDD, parallelism); + return findMatchingFilesForRecordKeys(partitionToFileInfo, + partitionRecordKeyPairRDD, parallelism, metaClient); } /** @@ -232,16 +233,18 @@ private int determineParallelism(int inputParallelism, int totalSubPartitions) { * Load all involved files as pair RDD. */ @VisibleForTesting - List> loadInvolvedFiles(List partitions, - final HoodieTable hoodieTable) { + List> loadInvolvedFiles(List partitions, final JavaSparkContext jsc, + final HoodieTableMetaClient metaClient) { // Obtain the latest data files from all the partitions. List> dataFilesList = jsc .parallelize(partitions, Math.max(partitions.size(), 1)).flatMapToPair(partitionPath -> { - java.util.Optional latestCommitTime = hoodieTable.getCommitsTimeline() + java.util.Optional latestCommitTime = metaClient.getCommitsTimeline() .filterCompletedInstants().lastInstant(); List> filteredFiles = new ArrayList<>(); if (latestCommitTime.isPresent()) { - filteredFiles = hoodieTable.getROFileSystemView() + TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(metaClient, metaClient + .getCommitsTimeline().filterCompletedInstants()); + filteredFiles = fileSystemView .getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp()) .map(f -> new Tuple2<>(partitionPath, f)).collect(toList()); } @@ -253,7 +256,7 @@ List> loadInvolvedFiles(List partitio return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1)).mapToPair(ft -> { try { String[] minMaxKeys = ParquetUtils - .readMinMaxRecordKeys(hoodieTable.getHadoopConf(), ft._2().getFileStatus().getPath()); + .readMinMaxRecordKeys(metaClient.getHadoopConf(), ft._2().getFileStatus().getPath()); return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1])); } catch (MetadataNotFoundException me) { @@ -352,9 +355,9 @@ JavaPairRDD> explodeRecordRDDWithFileCompariso * Make sure the parallelism is atleast the groupby parallelism for tagging location */ @VisibleForTesting - JavaPairRDD findMatchingFilesForRecordKeys(HoodieTable hoodieTable, + JavaPairRDD findMatchingFilesForRecordKeys( final Map> partitionToFileIndexInfo, - JavaPairRDD partitionRecordKeyPairRDD, int totalSubpartitions) { + JavaPairRDD partitionRecordKeyPairRDD, int totalSubpartitions, HoodieTableMetaClient metaClient) { int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), totalSubpartitions); @@ -367,7 +370,7 @@ JavaPairRDD findMatchingFilesForRecordKeys(HoodieTable hoodieTab .sortByKey(true, joinParallelism); return fileSortedTripletRDD.mapPartitionsWithIndex( - new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true) + new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true) .flatMap(indexLookupResults -> indexLookupResults.iterator()) .filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0) .flatMapToPair(lookupResult -> { @@ -410,8 +413,8 @@ record = new HoodieRecord(record.getKey(), record.getData()); } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieTable hoodieTable) { + public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, + HoodieTableMetaClient metaClient) { return writeStatusRDD; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java index 074ec56dae86..2d6ad74aef06 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java @@ -20,11 +20,11 @@ import com.uber.hoodie.common.BloomFilter; 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.exception.HoodieException; import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.func.LazyIterableIterator; -import com.uber.hoodie.table.HoodieTable; import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -48,10 +48,10 @@ public class HoodieBloomIndexCheckFunction implements private final String basePath; - private final HoodieTable table; + private final HoodieTableMetaClient metaClient; - public HoodieBloomIndexCheckFunction(HoodieTable table, String basePath) { - this.table = table; + public HoodieBloomIndexCheckFunction(HoodieTableMetaClient metaClient, String basePath) { + this.metaClient = metaClient; this.basePath = basePath; } @@ -121,7 +121,7 @@ private void initState(String fileName, String partitionPath) throws HoodieIndex try { Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName); bloomFilter = ParquetUtils - .readBloomFilterFromParquetMetadata(table.getHadoopConf(), filePath); + .readBloomFilterFromParquetMetadata(metaClient.getHadoopConf(), filePath); candidateRecordKeys = new ArrayList<>(); currentFile = fileName; currentParitionPath = partitionPath; @@ -169,7 +169,7 @@ protected List computeNext() { .debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); } ret.add(new IndexLookupResult(currentFile, - checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath))); + checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath))); initState(fileName, partitionPath); if (bloomFilter.mightContain(recordKey)) { @@ -182,7 +182,7 @@ protected List computeNext() { } } - // 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()) { Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile); logger.info( @@ -192,7 +192,7 @@ protected List computeNext() { logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); } ret.add(new IndexLookupResult(currentFile, - checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath))); + checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath))); } } catch (Throwable e) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java index a8e3921730c6..ba80ce31b689 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bucketed/BucketedIndex.java @@ -24,10 +24,10 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.index.HoodieIndex; -import com.uber.hoodie.table.HoodieTable; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaPairRDD; @@ -59,13 +59,14 @@ private String getBucket(String recordKey) { @Override public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, - HoodieTable table) { + JavaSparkContext jsc, HoodieTableMetaClient metaClient) { return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey())))); } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieTable hoodieTable) throws HoodieIndexException { + public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, + HoodieTableMetaClient metaClient) + throws HoodieIndexException { return recordRDD.map(record -> { String bucket = getBucket(record.getRecordKey()); //HACK(vc) a non-existent commit is provided here. @@ -75,8 +76,9 @@ public JavaRDD> tagLocation(JavaRDD> recordRDD, } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieTable hoodieTable) throws HoodieIndexException { + public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, + HoodieTableMetaClient metaClient) + throws HoodieIndexException { return writeStatusRDD; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java index 4928d144055d..547a2091f335 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java @@ -25,13 +25,13 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; 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.timeline.HoodieInstant; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException; import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.index.HoodieIndex; -import com.uber.hoodie.table.HoodieTable; import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; @@ -77,7 +77,7 @@ public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) { @Override public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, - HoodieTable table) { + JavaSparkContext jsc, HoodieTableMetaClient metaClient) { //TODO : Change/Remove filterExists in HoodieReadClient() and revisit throw new UnsupportedOperationException("HBase index does not implement check exist"); } @@ -119,8 +119,8 @@ private Get generateStatement(String key) throws IOException { .addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN); } - private boolean checkIfValidCommit(HoodieTable hoodieTable, String commitTs) { - HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline(); + private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) { + HoodieTimeline commitTimeline = metaClient.getActiveTimeline().filterCompletedInstants(); // 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 return !commitTimeline.empty() && (commitTimeline @@ -133,8 +133,8 @@ private boolean checkIfValidCommit(HoodieTable hoodieTable, String commitTs) /** * Function that tags each HoodieRecord with an existing location, if known. */ - private Function2>, Iterator>> - locationTagFunction(HoodieTable hoodieTable) { + private Function2>, + Iterator>> locationTagFunction(HoodieTableMetaClient metaClient) { return (Function2>, Iterator>>) (partitionNum, hoodieRecordIterator) -> { @@ -176,7 +176,7 @@ private boolean checkIfValidCommit(HoodieTable hoodieTable, String commitTs) String partitionPath = Bytes .toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN)); - if (checkIfValidCommit(hoodieTable, commitTs)) { + if (checkIfValidCommit(metaClient, commitTs)) { currentRecord = new HoodieRecord( new HoodieKey(currentRecord.getRecordKey(), partitionPath), currentRecord.getData()); @@ -211,13 +211,12 @@ private boolean checkIfValidCommit(HoodieTable hoodieTable, String commitTs) } @Override - public JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieTable hoodieTable) { - return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable), true); + public JavaRDD> tagLocation(JavaRDD> recordRDD, JavaSparkContext jsc, + HoodieTableMetaClient metaClient) { + return recordRDD.mapPartitionsWithIndex(locationTagFunction(metaClient), true); } - private Function2, Iterator> - updateLocationFunction() { + private Function2, Iterator> updateLocationFunction() { return (Function2, Iterator>) (partition, statusIterator) -> { Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize(); @@ -306,8 +305,8 @@ private void doPutsAndDeletes(HTable hTable, List puts, List delete } @Override - public JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieTable hoodieTable) { + public JavaRDD updateLocation(JavaRDD writeStatusRDD, JavaSparkContext jsc, + HoodieTableMetaClient metaClient) { return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 7ead846deacf..fd482ecebe74 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -31,7 +31,6 @@ 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.HoodieLogBlock; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.config.HoodieWriteConfig; @@ -45,6 +44,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -60,21 +60,35 @@ public class HoodieAppendHandle extends HoodieIOHandle { private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class); + // This acts as the sequenceID for records written private static AtomicLong recordIndex = new AtomicLong(1); private final WriteStatus writeStatus; private final String fileId; + // Buffer for holding records in memory before they are flushed to disk List recordList = new ArrayList<>(); + // Buffer for holding records (to be deleted) in memory before they are flushed to disk List keysToDelete = new ArrayList<>(); private TableFileSystemView.RealtimeView fileSystemView; private String partitionPath; private Iterator> recordItr; + // Total number of records written during an append private long recordsWritten = 0; + // Total number of records deleted during an append 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 HoodieLogFile currentLogFile; private Writer writer; + // Flag used to initialize some metadata private boolean doInit = true; + // Total number of bytes written during this append phase (an estimation) 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 header = Maps.newHashMap(); public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String fileId, Iterator> recordItr) { @@ -87,43 +101,46 @@ public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTab this.recordItr = recordItr; } - private void init(String partitionPath) { - - // extract some information from the first record - FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)).findFirst() - .get(); - // HACK(vc) This also assumes a base file. It will break, if appending without one. - String latestValidFilePath = fileSlice.getDataFile().get().getFileName(); - String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath); - writeStatus.getStat().setPrevCommit(baseCommitTime); - writeStatus.setFileId(fileId); - writeStatus.setPartitionPath(partitionPath); - writeStatus.getStat().setFileId(fileId); - this.partitionPath = partitionPath; + public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { + this(config, commitTime, hoodieTable, UUID.randomUUID().toString(), null); + } - try { - this.writer = HoodieLogFormat.newWriterBuilder() - .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) - .withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion( - fileSlice.getLogFiles().map(logFile -> logFile.getLogVersion()) - .max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) - .withSizeThreshold(config.getLogFileMaxSize()).withFs(fs) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); - this.currentLogFile = writer.getLogFile(); - ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion()); - ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize()); - } catch (Exception e) { - logger.error("Error in update task at commit " + commitTime, e); - writeStatus.setGlobalError(e); - throw new HoodieUpsertException( - "Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit " - + commitTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() - + partitionPath, e); + private void init(HoodieRecord record) { + if (doInit) { + this.partitionPath = record.getPartitionPath(); + // extract some information from the first record + Optional fileSlice = fileSystemView.getLatestFileSlices(partitionPath) + .filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)).findFirst(); + String baseCommitTime = commitTime; + if (fileSlice.isPresent()) { + baseCommitTime = fileSlice.get().getBaseInstantForLogAppend(); + } else { + // This means there is no base data file, start appending to a new log file + fileSlice = Optional.of(new FileSlice(baseCommitTime, this.fileId)); + logger.info("New InsertHandle for partition :" + partitionPath); + } + writeStatus.getStat().setPrevCommit(baseCommitTime); + writeStatus.setFileId(fileId); + writeStatus.setPartitionPath(partitionPath); + writeStatus.getStat().setFileId(fileId); + averageRecordSize = SizeEstimator.estimate(record); + try { + this.writer = createLogWriter(fileSlice, baseCommitTime); + this.currentLogFile = writer.getLogFile(); + ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion()); + ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize()); + } catch (Exception e) { + logger.error("Error in update task at commit " + commitTime, e); + writeStatus.setGlobalError(e); + throw new HoodieUpsertException( + "Failed to initialize HoodieAppendHandle for FileId: " + fileId + " on commit " + + commitTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() + + 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 getIndexedRecord(HoodieRecord hoodieRecord) { @@ -160,38 +177,11 @@ private Optional getIndexedRecord(HoodieRecord hoodieRecord) { // 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 public void doAppend() { - - int maxBlockSize = config.getLogFileDataBlockMaxSize(); - int numberOfRecords = 0; - Map header = Maps.newHashMap(); - header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime); - header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); while (recordItr.hasNext()) { HoodieRecord record = recordItr.next(); - // update the new location of the record, so we know where to find it next - record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); - if (doInit) { - 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 = getIndexedRecord(record); - if (indexedRecord.isPresent()) { - recordList.add(indexedRecord.get()); - } else { - keysToDelete.add(record.getRecordKey()); - } - numberOfRecords++; + init(record); + flushToDiskIfRequired(record); + writeToBuffer(record); } doAppend(header); estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; @@ -199,6 +189,8 @@ public void doAppend() { private void doAppend(Map header) { try { + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); if (recordList.size() > 0) { writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, header)); recordList.clear(); @@ -214,11 +206,37 @@ private void doAppend(Map header) { } } - public void close() { + @Override + public boolean canWrite(HoodieRecord record) { + return config.getParquetMaxFileSize() >= estimatedNumberOfBytesWritten * config + .getLogFileToParquetCompressionRatio(); + } + + @Override + public void write(HoodieRecord record, Optional insertValue) { + Optional recordMetadata = record.getData().getMetadata(); 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) { writer.close(); } + writeStatus.getStat().setPrevCommit(commitTime); + writeStatus.getStat().setFileId(this.fileId); writeStatus.getStat().setNumWrites(recordsWritten); writeStatus.getStat().setNumDeletes(recordsDeleted); writeStatus.getStat().setTotalWriteBytes(estimatedNumberOfBytesWritten); @@ -226,13 +244,54 @@ public void close() { RuntimeStats runtimeStats = new RuntimeStats(); runtimeStats.setTotalUpsertTime(timer.endTimer()); writeStatus.getStat().setRuntimeStats(runtimeStats); + return writeStatus; } catch (IOException e) { throw new HoodieUpsertException("Failed to close UpdateHandle", e); } } + @Override public WriteStatus getWriteStatus() { return writeStatus; } + private Writer createLogWriter(Optional 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 record) { + // update the new location of the record, so we know where to find it next + record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); + Optional 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; + } + } + } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java index c748dfe849ab..2f7006fe4e57 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java @@ -38,7 +38,7 @@ /** * Cleaner is responsible for garbage collecting older files in a given partition path, such that - *

1) It provides sufficient time for existing queries running on older versions, to finish

+ *

1) It provides sufficient time for existing queries running on older versions, to close

* 2) It bounds the growth of the files in the file system

TODO: Should all cleaning be done * based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata} */ diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index 10d9f94cc001..bf78467ffe38 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -52,6 +52,7 @@ import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; /** * Archiver to bound the growth of .commit files @@ -99,9 +100,9 @@ private void close() { /** * Check if commits need to be archived. If yes, archive commits. */ - public boolean archiveIfRequired() { + public boolean archiveIfRequired(final JavaSparkContext jsc) { try { - List instantsToArchive = getInstantsToArchive().collect(Collectors.toList()); + List instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList()); boolean success = true; if (instantsToArchive.iterator().hasNext()) { this.writer = openWriter(); @@ -117,13 +118,13 @@ public boolean archiveIfRequired() { } } - private Stream getInstantsToArchive() { + private Stream getInstantsToArchive(JavaSparkContext jsc) { // TODO : rename to max/minInstantsToKeep int maxCommitsToKeep = config.getMaxCommitsToKeep(); 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 // TODO: Handle ROLLBACK_ACTION in future diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index 0384db71c680..e05e34d2fef3 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -31,8 +31,8 @@ import com.uber.hoodie.io.storage.HoodieStorageWriterFactory; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; +import java.util.Iterator; import java.util.Optional; -import java.util.UUID; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -49,12 +49,13 @@ public class HoodieCreateHandle extends HoodieIOH private Path tempPath = null; private long recordsWritten = 0; private long recordsDeleted = 0; + private Iterator> recordIterator; public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, - String partitionPath) { + String partitionPath, String fileId) { super(config, commitTime, hoodieTable); this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName()); - status.setFileId(UUID.randomUUID().toString()); + status.setFileId(fileId); status.setPartitionPath(partitionPath); final int sparkPartitionId = TaskContext.getPartitionId(); @@ -77,12 +78,13 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTab logger.info("New InsertHandle for partition :" + partitionPath); } - /** - * Determines whether we can accept the incoming records, into the current file, depending on - *

- * - Whether it belongs to the same partitionPath as existing records - Whether the current file - * written bytes lt max file size - */ + public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String partitionPath, String fileId, Iterator> recordIterator) { + this(config, commitTime, hoodieTable, partitionPath, fileId); + this.recordIterator = recordIterator; + } + + @Override public boolean canWrite(HoodieRecord record) { return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath()); } @@ -114,9 +116,30 @@ public void write(HoodieRecord record, Optional avroRecord) { } } + /** + * Writes all records passed + */ + public void write() { + try { + while (recordIterator.hasNext()) { + HoodieRecord 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 */ + @Override public WriteStatus close() { logger.info("Closing the file " + status.getFileId() + " as we are done with all the records " + recordsWritten); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index e4a43af1998b..cc8272283415 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -16,6 +16,8 @@ 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.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -26,7 +28,9 @@ import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; +import java.util.Optional; import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -103,4 +107,29 @@ public Path makeTempPath(String partitionPath, int taskPartitionId, String fileN public Schema getSchema() { return schema; } + + /** + * Determines whether we can accept the incoming records, into the current file, depending on + *

+ * - 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 insertValue) { + } + + public WriteStatus close() { + return null; + } + + public WriteStatus getWriteStatus() { + return null; + } + } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 679d5fa8e1cc..fbc2b2bf5b8e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -243,7 +243,8 @@ public void write(GenericRecord oldRecord) { } } - public void close() { + @Override + public WriteStatus close() { try { // write out any pending records (this can happen when inserts are turned into updates) Iterator pendingRecordsItr = keyToNewRecords.keySet().iterator(); @@ -269,6 +270,7 @@ public void close() { RuntimeStats runtimeStats = new RuntimeStats(); runtimeStats.setTotalUpsertTime(timer.endTimer()); writeStatus.getStat().setRuntimeStats(runtimeStats); + return writeStatus; } catch (IOException e) { throw new HoodieUpsertException("Failed to close UpdateHandle", e); } @@ -283,6 +285,7 @@ private Path getStorageWriterPath() { return (this.tempPath == null) ? this.newFilePath : this.tempPath; } + @Override public WriteStatus getWriteStatus() { return writeStatus; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java index 8180f7dc18a4..3d5ffc40cb38 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/CompactionOperation.java @@ -18,11 +18,13 @@ import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.io.compact.strategy.CompactionStrategy; import java.io.Serializable; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -33,10 +35,10 @@ */ public class CompactionOperation implements Serializable { - private String dataFileCommitTime; - private long dataFileSize; + private Optional dataFileCommitTime; + private Optional dataFileSize; private List deltaFilePaths; - private String dataFilePath; + private Optional dataFilePath; private String fileId; private String partitionPath; private Map metrics; @@ -46,24 +48,32 @@ public class CompactionOperation implements Serializable { public CompactionOperation() { } - public CompactionOperation(HoodieDataFile dataFile, String partitionPath, + public CompactionOperation(Optional dataFile, String partitionPath, List logFiles, HoodieWriteConfig writeConfig) { - this.dataFilePath = dataFile.getPath(); - this.fileId = dataFile.getFileId(); + if (dataFile.isPresent()) { + 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.dataFileCommitTime = dataFile.getCommitTime(); - this.dataFileSize = dataFile.getFileSize(); this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()) .collect(Collectors.toList()); this.metrics = writeConfig.getCompactionStrategy() - .captureMetrics(dataFile, partitionPath, logFiles); + .captureMetrics(writeConfig, dataFile, partitionPath, logFiles); } - public String getDataFileCommitTime() { + public Optional getDataFileCommitTime() { return dataFileCommitTime; } - public long getDataFileSize() { + public Optional getDataFileSize() { return dataFileSize; } @@ -71,7 +81,7 @@ public List getDeltaFilePaths() { return deltaFilePaths; } - public String getDataFilePath() { + public Optional getDataFilePath() { return dataFilePath; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index e3d00f75e4a3..b8fbc72b89c8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -86,16 +86,19 @@ public JavaRDD compact(JavaSparkContext jsc, HoodieWriteConfig conf private JavaRDD executeCompaction(JavaSparkContext jsc, List operations, HoodieTable hoodieTable, HoodieWriteConfig config, 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"); 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()); } - private List compact(HoodieTable hoodieTable, HoodieWriteConfig config, + private List compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient, + HoodieWriteConfig config, CompactionOperation operation, String commitTime) throws IOException { - FileSystem fs = hoodieTable.getMetaClient().getFs(); + FileSystem fs = metaClient.getFs(); Schema readerSchema = HoodieAvroUtils .addMetadataFields(new Schema.Parser().parse(config.getSchema())); @@ -107,7 +110,6 @@ private List compact(HoodieTable hoodieTable, HoodieWriteConfig con // Load all the delta commits since the last compaction commit and get all the blocks to be // loaded and load it using CompositeAvroLogReader // Since a DeltaCommit is not defined yet, reading all the records. revisit this soon. - HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); String maxInstantTime = metaClient.getActiveTimeline() .getTimelineOfActions( Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, @@ -125,9 +127,16 @@ private List compact(HoodieTable hoodieTable, HoodieWriteConfig con } // Compacting is very similar to applying updates to existing file - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); - Iterator> result = table - .handleUpdate(commitTime, operation.getFileId(), scanner.getRecords()); + Iterator> result; + // If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a + // 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> resultIterable = () -> result; return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream) .map(s -> { @@ -173,10 +182,10 @@ private List getCompactionWorkload(JavaSparkContext jsc, .getLatestFileSlices(partitionPath).map( s -> { List logFiles = s.getLogFiles().sorted(HoodieLogFile - .getLogVersionComparator().reversed()).collect(Collectors.toList()); + .getBaseInstantAndLogVersionComparator().reversed()).collect(Collectors.toList()); totalLogFiles.add((long) logFiles.size()); 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()) .collect(toList()).iterator()).collect(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java index 75d62e803368..2a7e659545b4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java @@ -54,16 +54,19 @@ public abstract class CompactionStrategy implements Serializable { * @param logFiles - List of log files to compact with the base file * @return Map[String, Object] - metrics captured */ - public Map captureMetrics(HoodieDataFile dataFile, String partitionPath, - List logFiles) { + public Map captureMetrics(HoodieWriteConfig writeConfig, Optional dataFile, String + partitionPath, List logFiles) { Map metrics = Maps.newHashMap(); + Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize(); // Total size of all the log files Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(Optional::isPresent) .map(Optional::get).reduce((size1, size2) -> size1 + size2).orElse(0L); // 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 - 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 Long totalIO = totalIORead + totalIOWrite; // Save these metrics and we will use during the filter diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java index a7f729e36cf9..39f66b2fe528 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java @@ -39,10 +39,11 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE"; @Override - public Map captureMetrics(HoodieDataFile dataFile, String partitionPath, + public Map captureMetrics(HoodieWriteConfig config, Optional dataFile, String + partitionPath, List logFiles) { - Map metrics = super.captureMetrics(dataFile, partitionPath, logFiles); + Map metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles); // Total size of all the log files Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize) .filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java index edd2c062038e..bc8b491703e2 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java @@ -72,7 +72,7 @@ public HoodieParquetWriter(String commitTime, Path file, HoodieParquetConfig par this.schema = schema; } - private static Configuration registerFileSystem(Path file, Configuration conf) { + public static Configuration registerFileSystem(Path file, Configuration conf) { Configuration returnConf = new Configuration(conf); String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl", diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java index a5430cf89f63..2e83108e434b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieStorageWriterFactory.java @@ -30,18 +30,17 @@ public class HoodieStorageWriterFactory { - public static HoodieStorageWriter - getStorageWriter(String commitTime, Path path, HoodieTable hoodieTable, + public static HoodieStorageWriter getStorageWriter( + String commitTime, Path path, HoodieTable hoodieTable, HoodieWriteConfig config, Schema schema) throws IOException { //TODO - based on the metadata choose the implementation of HoodieStorageWriter // Currently only parquet is supported return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable); } - private static HoodieStorageWriter - newParquetStorageWriter( - String commitTime, Path path, HoodieWriteConfig config, Schema schema, - HoodieTable hoodieTable) throws IOException { + private static HoodieStorageWriter newParquetStorageWriter(String commitTime, Path path, + HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable) throws IOException { BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP()); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/log/HoodieLogWriter.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/log/HoodieLogWriter.java new file mode 100644 index 000000000000..df7e4591044a --- /dev/null +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/log/HoodieLogWriter.java @@ -0,0 +1,117 @@ +/* + * 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.io.storage.log; +// +//import com.uber.hoodie.avro.HoodieAvroWriteSupport; +//import com.uber.hoodie.common.model.FileSlice; +//import com.uber.hoodie.common.model.HoodieLogFile; +//import com.uber.hoodie.common.model.HoodieRecord; +//import com.uber.hoodie.common.model.HoodieRecordLocation; +//import com.uber.hoodie.common.model.HoodieRecordPayload; +//import com.uber.hoodie.common.table.log.HoodieLogFormat; +//import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer; +//import com.uber.hoodie.common.table.log.HoodieLogFormatWriter; +//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.config.HoodieWriteConfig; +//import com.uber.hoodie.io.storage.HoodieParquetConfig; +//import com.uber.hoodie.io.storage.HoodieParquetWriter; +//import com.uber.hoodie.io.storage.HoodieStorageWriter; +//import com.uber.hoodie.io.storage.HoodieWrapperFileSystem; +//import java.io.IOException; +//import java.util.ArrayList; +//import java.util.Comparator; +//import java.util.List; +//import java.util.Optional; +//import java.util.concurrent.atomic.AtomicLong; +//import org.apache.avro.Schema; +//import org.apache.avro.generic.GenericRecord; +//import org.apache.avro.generic.IndexedRecord; +//import org.apache.hadoop.conf.Configuration; +//import org.apache.hadoop.fs.Path; +//import org.apache.parquet.hadoop.ParquetFileWriter; +//import org.apache.parquet.hadoop.ParquetWriter; +//import org.apache.spark.TaskContext; +// +///** +// * HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides +// * a way to check if the current file can take more records with the canWrite() +// */ +//public class HoodieLogWriter +// implements HoodieStorageWriter { +// +// private final Path file; +// private final HoodieWrapperFileSystem fs; +// private final long maxFileSize; +// private final HoodieAvroWriteSupport writeSupport; +// private final String commitTime; +// private final Schema schema; +// private HoodieLogFormatWriter writer; +// private HoodieWriteConfig config; +// +// +// public HoodieLogWriter(String commitTime, Path file, +// Schema schema, Configuration configuration, HoodieWriteConfig config) throws IOException, InterruptedException { +// this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, configuration); +// this.fs = (HoodieWrapperFileSystem) this.file +// .getFileSystem(HoodieParquetWriter.registerFileSystem(file, configuration)); +// // We cannot accurately measure the snappy compressed output file size. We are choosing a +// // conservative 10% +// // TODO - compute this compression ratio dynamically by looking at the bytes written to the +// // stream and the actual file size reported by HDFS +// this.maxFileSize = config.getParquetMaxFileSize() + Math +// .round(config.getParquetBlockSize() * config.getParquetCompressionRatio()); +// this.writeSupport = null; +// this.commitTime = commitTime; +// this.schema = schema; +// this.writer = createLogWriter(0, ""); +// this.config = config; +// } +// +// @Override +// public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException { +// throw new UnsupportedOperationException("Log Block implementation"); +// } +// +// public boolean canWrite() { +// return fs.getBytesWritten(file) < maxFileSize; +// } +// +// @Override +// public void writeAvro(String key, IndexedRecord object) throws IOException { +// } +// +// public void writeLogBlock(HoodieLogBlock hoodieLogBlock) throws IOException, InterruptedException { +// this.writer.appendBlock(hoodieLogBlock); +// } +// +// private HoodieLogFormatWriter createLogWriter(int logVersion, String baseCommitTime) +// throws IOException, InterruptedException { +// return HoodieLogFormat.newWriterBuilder() +// .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath)) +// .withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(logVersion) +// .withSizeThreshold(config.getLogFileMaxSize()).withFs(fs) +// .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); +// } +// +// @Override +// public void close() { +// +// } +// +//} diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index ab194c4f52b7..9e6850bed8bd 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -40,10 +40,11 @@ import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieNotSupportedException; 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.SparkBoundedInMemoryExecutor; import com.uber.hoodie.io.HoodieCleanHelper; +import com.uber.hoodie.io.HoodieCreateHandle; import com.uber.hoodie.io.HoodieMergeHandle; import java.io.IOException; import java.io.Serializable; @@ -64,6 +65,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroParquetReader; @@ -90,8 +92,8 @@ public class HoodieCopyOnWriteTable extends Hoodi private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class); - public HoodieCopyOnWriteTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { - super(config, metaClient); + public HoodieCopyOnWriteTable(HoodieWriteConfig config, JavaSparkContext jsc) { + super(config, jsc); } private static PairFlatMapFunction>, String, @@ -225,7 +227,15 @@ protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, public Iterator> handleInsert(String commitTime, Iterator> recordItr) throws Exception { - return new LazyInsertIterable<>(recordItr, config, commitTime, this); + return new CopyOnWriteLazyInsertIterable<>(recordItr, config, commitTime, this); + } + + public Iterator> handleInsert(String commitTime, String partitionPath, String fileId, + Iterator> recordItr) { + HoodieCreateHandle createHandle = new HoodieCreateHandle(config, commitTime, this, partitionPath, fileId, + recordItr); + createHandle.write(); + return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator(); } @SuppressWarnings("unchecked") @@ -289,17 +299,34 @@ public List clean(JavaSparkContext jsc) { */ protected Map deleteCleanedFiles(String partitionPath, List commits) throws IOException { + Map results = Maps.newHashMap(); + deleteCleanedFiles(results, partitionPath, commits); + return results; + } + + /** + * Common method used for cleaning out parquet files under a partition path during rollback of a + * set of commits + */ + protected Map deleteCleanedFiles(Map results, String partitionPath, + List commits) + throws IOException { logger.info("Cleaning path " + partitionPath); FileSystem fs = getMetaClient().getFs(); - FileStatus[] toBeDeleted = fs - .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 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); + // TODO (NA) : Move log files logic out of CopyOnWrite + } 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; + }; + FileStatus[] toBeDeleted = fs.listStatus(new Path(config.getBasePath(), partitionPath), filter); for (FileStatus file : toBeDeleted) { boolean success = fs.delete(file.getPath(), false); results.put(file, success); @@ -311,7 +338,7 @@ protected Map deleteCleanedFiles(String partitionPath, List @Override public List rollback(JavaSparkContext jsc, List commits) throws IOException { - String actionType = this.getCommitActionType(); + String actionType = metaClient.getCommitActionType(); HoodieActiveTimeline activeTimeline = this.getActiveTimeline(); List inflights = this.getInflightCommitTimeline().getInstants() .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 857402564456..395c1394c6ee 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -27,11 +27,12 @@ import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; 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.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.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.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; @@ -39,7 +40,11 @@ import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.exception.HoodieUpsertException; +<<<<<<< 23d53763c4db744fe936e2e609fe6abaf7bd9a82 import com.uber.hoodie.index.HoodieIndex; +======= +import com.uber.hoodie.func.MergeOnReadLazyInsertIterable; +>>>>>>> Adding ability for inserts to be written to log files import com.uber.hoodie.io.HoodieAppendHandle; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; import java.io.IOException; @@ -78,8 +83,8 @@ public class HoodieMergeOnReadTable extends // UpsertPartitioner for MergeOnRead table type private MergeOnReadUpsertPartitioner mergeOnReadUpsertPartitioner; - public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { - super(config, metaClient); + public HoodieMergeOnReadTable(HoodieWriteConfig config, JavaSparkContext jsc) { + super(config, jsc); } @Override @@ -110,6 +115,17 @@ public Iterator> handleUpdate(String commitTime, String fileId } } + @Override + public Iterator> handleInsert(String commitTime, + Iterator> recordItr) throws Exception { + // If canIndeLogFiles, 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 public JavaRDD compact(JavaSparkContext jsc, String compactionCommitTime) { logger.info("Checking if compaction needs to be run on " + config.getBasePath()); @@ -180,29 +196,41 @@ public List rollback(JavaSparkContext jsc, List comm case HoodieTimeline.DELTA_COMMIT_ACTION: try { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( - this.getCommitTimeline().getInstantDetails( + metaClient.getCommitTimeline().getInstantDetails( new HoodieInstant(true, instant.getAction(), instant.getTimestamp())) .get()); // read commit file and (either append delete blocks or delete file) - Map filesToDeletedStatus = new HashMap<>(); + final Map filesToDeletedStatus = new HashMap<>(); Map filesToNumBlocksRollback = new HashMap<>(); - // we do not know fileIds for inserts (first inserts are parquet files), delete - // all parquet files for the corresponding failed commit, if present (same as COW) - filesToDeletedStatus = super - .deleteCleanedFiles(partitionPath, Arrays.asList(commit)); + // In case all data was inserts and the commit failed, there is no partition stats + if (commitMetadata.getPartitionToWriteStats().size() == 0) { + super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, Arrays.asList(commit)); + } // append rollback blocks for updates if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { // This needs to be done since GlobalIndex at the moment does not store the latest commit time + // Also, the baseCommitTime changes if compaction has been requested so we need to do this anyways Map fileIdToLatestCommitTimeMap = hoodieIndex.isGlobal() ? this.getRTFileSystemView().getLatestFileSlices(partitionPath) - .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseCommitTime)) : null; + .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantForLogAppend)) : null; commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() .filter(wStat -> { - return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT - && wStat.getPrevCommit() != null; + if (wStat != null + && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT + && wStat.getPrevCommit() != 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, Arrays.asList(commit)); + } catch (IOException io) { + throw new UncheckedIOException(io); + } + return false; }).forEach(wStat -> { HoodieLogFormat.Writer writer = null; String baseCommitTime = wStat.getPrevCommit(); @@ -210,6 +238,10 @@ public List rollback(JavaSparkContext jsc, List comm baseCommitTime = fileIdToLatestCommitTimeMap.get(wStat.getFileId()); } 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 + // getBaseCommitTimeForAppendLog() writer = HoodieLogFormat.newWriterBuilder().onParentPath( new Path(this.getMetaClient().getBasePath(), partitionPath)) .withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime) @@ -217,15 +249,15 @@ public List rollback(JavaSparkContext jsc, List comm .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); Long numRollbackBlocks = 0L; // generate metadata - Map header = + Map header = Maps.newHashMap(); - header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, + header.put(HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); - header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, + header.put(HeaderMetadataType.TARGET_INSTANT_TIME, commit); - header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String + header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String .valueOf( - HoodieCommandBlock.HoodieCommandBlockTypeEnum + HoodieCommandBlockTypeEnum .ROLLBACK_PREVIOUS_BLOCK .ordinal())); // if update belongs to an existing log file @@ -298,6 +330,8 @@ protected List getSmallFiles(String partitionPath) { if (!commitTimeline.empty()) { HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); // find smallest file in partition and append to it + + // TODO - check if index.isglobal then small files are log files too Optional smallFileSlice = getRTFileSystemView() .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter( fileSlice -> fileSlice.getLogFiles().count() < 1 @@ -327,4 +361,5 @@ public List getSmallFileIds() { .collect(Collectors.toList()); } } + } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 6f62533c33f1..383743a22da8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -31,9 +31,9 @@ import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.exception.HoodieCommitException; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieSavepointException; +import com.uber.hoodie.index.HoodieIndex; import java.io.IOException; import java.io.Serializable; import java.util.Iterator; @@ -54,19 +54,21 @@ public abstract class HoodieTable implements Seri protected final HoodieWriteConfig config; protected final HoodieTableMetaClient metaClient; + protected final HoodieIndex index; - protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { + protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc) { this.config = config; - this.metaClient = metaClient; + this.metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true); + this.index = HoodieIndex.createIndex(config, jsc); } public static HoodieTable getHoodieTable( - HoodieTableMetaClient metaClient, HoodieWriteConfig config) { + HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) { switch (metaClient.getTableType()) { case COPY_ON_WRITE: - return new HoodieCopyOnWriteTable<>(config, metaClient); + return new HoodieCopyOnWriteTable<>(config, jsc); case MERGE_ON_READ: - return new HoodieMergeOnReadTable<>(config, metaClient); + return new HoodieMergeOnReadTable<>(config, jsc); default: throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); } @@ -124,21 +126,21 @@ public TableFileSystemView.RealtimeView getRTFileSystemView() { * Get the completed (commit + compaction) view of the file system for this table */ public TableFileSystemView getCompletedFileSystemView() { - return new HoodieTableFileSystemView(metaClient, getCommitsTimeline()); + return new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline()); } /** * Get only the completed (no-inflights) commit timeline */ public HoodieTimeline getCompletedCommitTimeline() { - return getCommitsTimeline().filterCompletedInstants(); + return metaClient.getCommitsTimeline().filterCompletedInstants(); } /** * Get only the inflights (no-completed) commit timeline */ public HoodieTimeline getInflightCommitTimeline() { - return getCommitsTimeline().filterInflights(); + return metaClient.getCommitsTimeline().filterInflights(); } /** @@ -190,49 +192,10 @@ public HoodieActiveTimeline getActiveTimeline() { } /** - * Get the commit timeline visible for this table + * Return the index */ - public HoodieTimeline getCommitsTimeline() { - switch (metaClient.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 :" + 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()); - } + public HoodieIndex getIndex() { + return index; } /** diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java index 086818274c93..80fdf602ab32 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -76,10 +76,10 @@ public class TestCleaner extends TestHoodieClientBase { /** * Helper method to do first batch of insert for clean by versions/commits tests * - * @param cfg Hoodie Write Config - * @param client Hoodie Client + * @param cfg Hoodie Write Config + * @param client Hoodie Client * @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 */ private void insertFirstBigBatchForClientCleanerTest( @@ -93,7 +93,6 @@ private void insertFirstBigBatchForClientCleanerTest( * (this is basically same as insert part of upsert, just adding it here so we can * catch breakages in insert(), if the implementation diverges.) */ - HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); String newCommitTime = client.startCommit(); List records = recordGenFunction.apply(newCommitTime, BIG_BATCH_INSERT_SIZE); @@ -108,7 +107,7 @@ private void insertFirstBigBatchForClientCleanerTest( HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); 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 - HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); assertFalse(table.getCompletedCommitTimeline().empty()); String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); @@ -116,7 +115,8 @@ private void insertFirstBigBatchForClientCleanerTest( assertEquals("The clean instant should be the same as the commit instant", commitTime, table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp()); - List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect(); + HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); + List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, metaClient).collect(); checkTaggedRecords(taggedRecords, newCommitTime); } @@ -158,10 +158,10 @@ public void testBulkInsertPreppedAndCleanByVersions() throws Exception { /** * Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective * - * @param insertFn Insert API to be tested - * @param upsertFn Upsert API to be tested + * @param insertFn Insert 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 - * 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 */ private void testInsertAndCleanByVersions( @@ -198,8 +198,8 @@ private void testInsertAndCleanByVersions( assertNoWriteErrors(statuses); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig()); - HoodieTimeline timeline = table.getCommitsTimeline(); + HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc); + HoodieTimeline timeline = metadata.getCommitsTimeline(); TableFileSystemView fsView = table.getFileSystemView(); // Need to ensure the following @@ -280,10 +280,10 @@ public void testBulkInsertAndCleanByCommits() throws Exception { /** * Test Helper for Cleaning by versions logic from HoodieWriteClient API perspective * - * @param insertFn Insert API to be tested - * @param upsertFn Upsert API to be tested + * @param insertFn Insert 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 - * 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 */ private void testInsertAndCleanByCommits( @@ -318,7 +318,7 @@ private void testInsertAndCleanByCommits( assertNoWriteErrors(statuses); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg); + HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg, jsc); HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline(); Optional earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); Set acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet()); @@ -365,7 +365,8 @@ public void testKeepLatestFileVersions() throws IOException { String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "000"); String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000"); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); List hoodieCleanStatsOne = table.clean(jsc); assertEquals("Must not clean any files", 0, @@ -377,7 +378,8 @@ public void testKeepLatestFileVersions() throws IOException { // make next commit, with 1 insert & 1 update per partition 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 file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert @@ -397,7 +399,7 @@ public void testKeepLatestFileVersions() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "002"); 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", file2P0C1); // update @@ -452,7 +454,7 @@ public void testKeepLatestFileVersionsMOR() throws IOException { // make 1 compaction commit HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); List hoodieCleanStats = table.clean(jsc); assertEquals("Must clean three files, one parquet and 2 log files", 3, getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); @@ -479,7 +481,8 @@ public void testKeepLatestCommits() throws IOException { String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "000"); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); List hoodieCleanStatsOne = table.clean(jsc); assertEquals("Must not clean any files", 0, @@ -492,7 +495,7 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 1 insert & 1 update per partition HoodieTestUtils.createCommitFiles(basePath, "001"); 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 file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_SECOND_PARTITION_PATH, "001"); // insert @@ -512,7 +515,7 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "002"); 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", file2P0C1); // update @@ -527,7 +530,7 @@ public void testKeepLatestCommits() throws IOException { // make next commit, with 2 updates to existing files, and 1 insert HoodieTestUtils.createCommitFiles(basePath, "003"); 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", file2P0C1); // update @@ -568,14 +571,15 @@ public void testCleanTemporaryDataFilesOnRollback() throws IOException { .withUseTempFolderCopyOnWriteForCreate(false) .withUseTempFolderCopyOnWriteForMerge(false).build(); 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()); assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true) .withUseTempFolderCopyOnWriteForMerge(false).build(); table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + config, jsc); table.rollback(jsc, Collections.emptyList()); assertEquals("All temp files are deleted.", 0, getTotalTempFiles()); } @@ -595,7 +599,8 @@ public void testCleaningWithZeroPartitonPaths() throws IOException { HoodieTestUtils.createCommitFiles(basePath, "000"); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); List hoodieCleanStatsOne = table.clean(jsc); assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty()); @@ -655,7 +660,8 @@ public void onTaskEnd(SparkListenerTaskEnd taskEnd) { updateAllFilesInPartition(filesP2C0, DEFAULT_THIRD_PARTITION_PATH, "003"); HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, + jsc); List hoodieCleanStats = table.clean(jsc); assertEquals(100, getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); @@ -677,7 +683,7 @@ public void onTaskEnd(SparkListenerTaskEnd taskEnd) { * Utility method to create temporary data files * * @param commitTime Commit Timestamp - * @param numFiles Number for files to be generated + * @param numFiles Number for files to be generated * @return generated files * @throws IOException in case of error */ diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java index 6d9dea7b4803..68e3e9f0b81e 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java @@ -96,7 +96,7 @@ public void testSavepointAndRollback() throws Exception { List partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); 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(); List dataFiles = partitionPaths.stream().flatMap(s -> { @@ -121,7 +121,7 @@ public void testSavepointAndRollback() throws Exception { assertNoWriteErrors(statuses); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, getConfig()); + table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView(); dataFiles = partitionPaths.stream().flatMap(s -> { @@ -142,7 +142,7 @@ public void testSavepointAndRollback() throws Exception { client.rollbackToSavepoint(savepoint.getTimestamp()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, getConfig()); + table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView(); dataFiles = partitionPaths.stream().flatMap(s -> { return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java index a9e374c40385..2d502d260dc5 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java @@ -184,9 +184,9 @@ private Function2, String, Integer> wrapRecordsGenFunctionFor final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc); List records = recordGenFunction.apply(commit, numRecords); final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true); - HoodieTable.getHoodieTable(metaClient, writeConfig); + HoodieTable.getHoodieTable(metaClient, writeConfig, jsc); JavaRDD taggedRecords = - index.tagLocation(jsc.parallelize(records, 1), HoodieTable.getHoodieTable(metaClient, writeConfig)); + index.tagLocation(jsc.parallelize(records, 1), jsc, metaClient); return taggedRecords.collect(); }; } @@ -348,7 +348,7 @@ JavaRDD writeBatch( fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); } 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 assertEquals("Incremental consumption from " + prevCommitTime diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 99790947e1ad..9fbd0871a41d 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -415,7 +415,7 @@ public void testSmallInsertHandlingForUpserts() throws Exception { assertEquals("2 files needs to be committed.", 2, statuses.size()); 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(); List files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) .collect(Collectors.toList()); @@ -519,7 +519,7 @@ public void testSmallInsertHandlingForInserts() throws Exception { assertEquals("2 files needs to be committed.", 2, statuses.size()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); List files = table.getROFileSystemView() .getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) .collect(Collectors.toList()); @@ -544,7 +544,7 @@ public void testCommitWritesRelativePaths() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); String commitTime = "000"; client.startCommitWithTime(commitTime); @@ -559,9 +559,9 @@ public void testCommitWritesRelativePaths() throws Exception { HoodieTestUtils.doesCommitExist(basePath, commitTime)); // Get parquet file paths from commit metadata - String actionType = table.getCommitActionType(); + String actionType = metaClient.getCommitActionType(); HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime); - HoodieTimeline commitTimeline = table.getCommitTimeline().filterCompletedInstants(); + HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants(); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(commitTimeline.getInstantDetails(commitInstant).get()); String basePath = table.getMetaClient().getBasePath(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java index d092c8304e04..a85bd3c23bff 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java @@ -28,7 +28,6 @@ import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieException; -import com.uber.hoodie.table.HoodieTable; import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; @@ -43,6 +42,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; @@ -152,14 +152,17 @@ public static Dataset readSince(String basePath, SQLContext sqlContext, Hoo /** * Reads the paths under the a hoodie dataset out as a DataFrame */ - public static Dataset read(String basePath, SQLContext sqlContext, FileSystem fs, String... paths) { + public static Dataset read(JavaSparkContext jsc, String basePath, SQLContext + sqlContext, + FileSystem + fs, String... + paths) { List filteredPaths = new ArrayList<>(); try { - HoodieTable hoodieTable = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); for (String path : paths) { TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView( - hoodieTable.getMetaClient(), hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path))); + metaClient, metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path))); List latestFiles = fileSystemView.getLatestDataFiles().collect(Collectors.toList()); for (HoodieDataFile file : latestFiles) { filteredPaths.add(file.getPath()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index 99af0e2e3164..83161aa2c320 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -213,10 +213,10 @@ public List generateUpdates(String commitTime, List /** * Generates new updates, randomly distributed across the keys above. There can be duplicates within the returned list + * * @param commitTime Commit Timestamp * @param n Number of updates (including dups) * @return list of hoodie record updates - * @throws IOException */ public List generateUpdates(String commitTime, Integer n) throws IOException { List updates = new ArrayList<>(); @@ -230,10 +230,10 @@ public List generateUpdates(String commitTime, Integer n) throws I /** * Generates deduped updates of keys previously inserted, randomly distributed across the keys above. + * * @param commitTime Commit Timestamp * @param n Number of unique records * @return list of hoodie record updates - * @throws IOException */ public List generateUniqueUpdates(String commitTime, Integer n) throws IOException { List updates = new ArrayList<>(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java index 385afaabfadc..b629b837fe53 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java @@ -16,7 +16,7 @@ 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.when; @@ -38,7 +38,7 @@ public class TestBoundedInMemoryExecutor { private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); private SparkBoundedInMemoryExecutor>, Integer> executor = null; + Tuple2>, Integer> executor = null; @After public void afterTest() { @@ -83,4 +83,4 @@ protected Integer getResult() { // There should be no remaining records in the buffer Assert.assertFalse(executor.isRemaining()); } -} +} \ No newline at end of file diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java index b555b8167247..4092f10a451a 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java @@ -16,7 +16,7 @@ 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.when; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java index dcd5a94a5eb4..20a14ea2c1c7 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java @@ -19,6 +19,7 @@ import static org.junit.Assert.fail; import com.uber.hoodie.WriteStatus; +import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.TestRawTripPayload; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; @@ -29,11 +30,14 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieCopyOnWriteTable; +import java.io.File; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaSparkContext; +import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -41,6 +45,7 @@ public class TestUpdateMapFunction { private String basePath = null; + private transient JavaSparkContext jsc = null; @Before public void init() throws Exception { @@ -49,6 +54,18 @@ public void init() throws Exception { folder.create(); this.basePath = folder.getRoot().getAbsolutePath(); 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 @@ -56,7 +73,7 @@ public void testSchemaEvolutionOnUpdate() throws Exception { // Create a bunch of records with a old version of schema HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt"); 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\"," + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; @@ -82,7 +99,7 @@ public void testSchemaEvolutionOnUpdate() throws Exception { String fileId = insertResult.next().get(0).getFileId(); System.out.println(fileId); - table = new HoodieCopyOnWriteTable(config, metaClient); + table = new HoodieCopyOnWriteTable(config, jsc); // New content with values for the newly added field recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}"; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java index eb4a878f903a..5fe7ceec131e 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHbaseIndex.java @@ -26,8 +26,7 @@ import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; -import com.uber.hoodie.common.model.HoodieTableType; -import com.uber.hoodie.common.table.HoodieTableConfig; +import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; @@ -74,7 +73,6 @@ public class TestHbaseIndex { private static String tableName = "test_table"; private String basePath = null; private transient FileSystem fs; - private HoodieTableMetaClient metaClient; public TestHbaseIndex() throws Exception { } @@ -117,9 +115,7 @@ public void before() throws Exception { folder.create(); basePath = folder.getRoot().getAbsolutePath(); // Initialize table - metaClient = HoodieTableMetaClient - .initTableType(utility.getConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE, tableName, - HoodieTableConfig.DEFAULT_PAYLOAD_CLASS); + HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); } @Test @@ -135,10 +131,11 @@ public void testSimpleTagLocationAndUpdate() throws Exception { HBaseIndex index = new HBaseIndex(config, jsc); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); 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 - JavaRDD javaRDD = index.tagLocation(writeRecords, hoodieTable); + JavaRDD javaRDD = index.tagLocation(writeRecords, jsc, metaClient); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); // Insert 200 records @@ -147,14 +144,14 @@ public void testSimpleTagLocationAndUpdate() throws Exception { // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed // commit - javaRDD = index.tagLocation(writeRecords, hoodieTable); + javaRDD = index.tagLocation(writeRecords, jsc, metaClient); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); // Now commit this & update location of records inserted and validate no errors writeClient.commit(newCommitTime, writeStatues); - // Now tagLocation for these records, hbaseIndex should tag them correctly - javaRDD = index.tagLocation(writeRecords, hoodieTable); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + javaRDD = index.tagLocation(writeRecords, jsc, metaClient); assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200); assertTrue(javaRDD.filter( @@ -175,8 +172,8 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { String newCommitTime = writeClient.startCommit(); List records = dataGen.generateInserts(newCommitTime, 200); JavaRDD writeRecords = jsc.parallelize(records, 1); - - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Insert 200 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); @@ -184,9 +181,8 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { // commit this upsert writeClient.commit(newCommitTime, writeStatues); - // Now tagLocation for these records, hbaseIndex should tag them - JavaRDD javaRDD = index.tagLocation(writeRecords, hoodieTable); + JavaRDD javaRDD = index.tagLocation(writeRecords, jsc, metaClient); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); // check tagged records are tagged with correct fileIds @@ -201,7 +197,7 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled // back commit - javaRDD = index.tagLocation(writeRecords, hoodieTable); + javaRDD = index.tagLocation(writeRecords, jsc, metaClient); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0); } @@ -228,15 +224,15 @@ public void testTotalGetsBatching() throws Exception { String newCommitTime = writeClient.startCommit(); List records = dataGen.generateInserts(newCommitTime, 250); JavaRDD writeRecords = jsc.parallelize(records, 1); - - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Insert 250 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); assertNoWriteErrors(writeStatues.collect()); // Now tagLocation for these records, hbaseIndex should tag them - index.tagLocation(writeRecords, hoodieTable); + index.tagLocation(writeRecords, jsc, metaClient); // 3 batches should be executed given batchSize = 100 and parallelism = 1 Mockito.verify(table, times(3)).get((List) anyObject()); @@ -255,8 +251,8 @@ public void testTotalPutsBatching() throws Exception { String newCommitTime = writeClient.startCommit(); List records = dataGen.generateInserts(newCommitTime, 250); JavaRDD writeRecords = jsc.parallelize(records, 1); - - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc); // Insert 200 records JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); @@ -276,7 +272,7 @@ public void testTotalPutsBatching() throws Exception { // Get all the files generated int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count(); - index.updateLocation(writeStatues, hoodieTable); + index.updateLocation(writeStatues, jsc, metaClient); // 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated, // so each fileId ideally gets updates Mockito.verify(table, atMost(numberOfDataFileIds)).put((List) anyObject()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java index 16f754a4bd49..002c1e4351ca 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHoodieIndex.java @@ -18,27 +18,58 @@ 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.HoodieWriteConfig; import com.uber.hoodie.index.bloom.HoodieBloomIndex; 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.rules.TemporaryFolder; 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 public void testCreateIndex() throws Exception { HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); // Different types - HoodieWriteConfig config = clientConfigBuilder.withPath("").withIndexConfig( + HoodieWriteConfig config = clientConfigBuilder.withPath(basePath).withIndexConfig( indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()).build(); - assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex); - config = clientConfigBuilder.withPath("") + assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HBaseIndex); + config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); - assertTrue(HoodieIndex.createIndex(config, null) instanceof InMemoryHashIndex); - config = clientConfigBuilder.withPath("") + assertTrue(HoodieIndex.createIndex(config, jsc) instanceof InMemoryHashIndex); + config = clientConfigBuilder.withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); - assertTrue(HoodieIndex.createIndex(config, null) instanceof HoodieBloomIndex); + assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HoodieBloomIndex); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java index 8e649c73c207..b9a49adf7910 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java @@ -95,6 +95,16 @@ public void init() throws IOException { 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 public void testLoadUUIDsInMemory() throws IOException { // Create one RDD of hoodie record @@ -171,8 +181,8 @@ public void testLoadInvolvedFiles() throws IOException { List partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metadata, config); - List> filesList = index.loadInvolvedFiles(partitions, table); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); + List> filesList = index.loadInvolvedFiles(partitions, jsc, metadata); // Still 0, as no valid commit assertEquals(filesList.size(), 0); @@ -181,7 +191,7 @@ public void testLoadInvolvedFiles() throws IOException { new File(basePath + "/.hoodie/20160401010101.commit").createNewFile(); new File(basePath + "/.hoodie/20150312101010.commit").createNewFile(); - filesList = index.loadInvolvedFiles(partitions, table); + filesList = index.loadInvolvedFiles(partitions, jsc, metadata); assertEquals(filesList.size(), 4); // these files will not have the key ranges assertNull(filesList.get(0)._2().getMaxRecordKey()); @@ -290,13 +300,13 @@ public void testTagLocationWithEmptyRDD() throws Exception { // Also create the metadata and config HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieTable table = HoodieTable.getHoodieTable(metadata, config); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); try { - bloomIndex.tagLocation(recordRDD, table); + bloomIndex.tagLocation(recordRDD, jsc, metadata); } catch (IllegalArgumentException e) { fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices " + "required"); } @@ -332,11 +342,11 @@ public void testTagLocation() throws Exception { // Also create the metadata and config HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieTable table = HoodieTable.getHoodieTable(metadata, config); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); - JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table); + JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, metadata); // Should not find any files for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -350,9 +360,9 @@ public void testTagLocation() throws Exception { // We do the tag again 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, metadata); // Check results for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -397,11 +407,11 @@ public void testCheckExists() throws Exception { // Also create the metadata and config HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - HoodieTable table = HoodieTable.getHoodieTable(metadata, config); + HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); - JavaPairRDD> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table); + JavaPairRDD> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, metadata); // Should not find any files for (Tuple2> record : taggedRecordRDD.collect()) { @@ -415,8 +425,7 @@ public void testCheckExists() throws Exception { // We do the tag again metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metadata, config); - taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table); + taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, metadata); // Check results for (Tuple2> record : taggedRecordRDD.collect()) { @@ -465,10 +474,10 @@ public void testBloomFilterFalseError() throws IOException, InterruptedException JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2)); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); 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); - JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table); + JavaRDD taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, metadata); // Check results for (HoodieRecord record : taggedRecordRDD.collect()) { @@ -525,14 +534,4 @@ private String writeParquetFile(String partitionPath, String filename, List updatedRecords = dataGen.generateUpdates(newCommitTime, records); JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); HoodieIndex index = new HoodieBloomIndex<>(config, jsc); - updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect(); + updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, metaClient).collect(); // Write them to corresponding avro logfiles HoodieTestUtils @@ -158,7 +158,7 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { // Verify that all data file has one log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) .collect(Collectors.toList()); @@ -169,7 +169,7 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { // Do a compaction metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); JavaRDD result = compactor .compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java index bbb826c10130..e5523f8beaad 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieMergeHandleDuplicateRecords.java @@ -247,7 +247,8 @@ private Dataset getRecords() { for (int i = 0; i < fullPartitionPaths.length; i++) { fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); } - Dataset dataSet = HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths); + Dataset dataSet = HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, + fullPartitionPaths); return dataSet; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java index c3aee4f2bb04..9965cfcba29f 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java @@ -30,6 +30,7 @@ import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Random; import java.util.stream.Collectors; import org.junit.Test; @@ -127,7 +128,7 @@ private List createCompactionOperations(HoodieWriteConfig c Map> sizesMap) { List operations = Lists.newArrayList(sizesMap.size()); 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)], v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), config)); }); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java index ebd31b25a13d..b9be42b71554 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java @@ -77,6 +77,7 @@ public void init() throws Exception { folder.create(); this.basePath = folder.getRoot().getAbsolutePath(); HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath); + } @Test @@ -90,9 +91,10 @@ public void testMakeNewPath() throws Exception { String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieWriteConfig config = makeHoodieClientConfig(); 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); assertTrue(newPath.toString().equals( this.basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, unitNumber, fileName))); @@ -117,7 +119,7 @@ public void testUpdateRecords() throws Exception { HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); 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) String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," @@ -187,7 +189,7 @@ public void testUpdateRecords() throws Exception { Thread.sleep(1000); String newCommitTime = HoodieTestUtils.makeNewCommitTime(); metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = new HoodieCopyOnWriteTable(config, metadata); + table = new HoodieCopyOnWriteTable(config, jsc); Iterator> iter = table .handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(), updatedRecords.iterator()); @@ -255,7 +257,7 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { String firstCommitTime = HoodieTestUtils.makeNewCommitTime(); 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) String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," @@ -290,7 +292,7 @@ public void testInsertWithPartialFailures() throws Exception { String commitTime = HoodieTestUtils.makeNewCommitTime(); FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); 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 // 10 records for partition 1, 1 record for partition 2. @@ -324,7 +326,7 @@ public void testInsertRecords() throws Exception { HoodieWriteConfig config = makeHoodieClientConfig(); String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); // Case 1: // 10 records for partition 1, 1 record for partition 2. @@ -372,7 +374,7 @@ public void testFileSizeUpsertRecords() throws Exception { .build()).build(); String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc); List records = new ArrayList<>(); // Approx 1150 records are written for block size of 64KB @@ -411,9 +413,9 @@ private List testUpsertPartitioner(int smal HoodieClientTestUtils.fakeDataFile(basePath, testPartitionPath, "001", "file1", fileSize); 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 insertRecords = dataGenerator.generateInserts("001", numInserts); List updateRecords = dataGenerator.generateUpdates("001", numUpdates); for (HoodieRecord updateRec : updateRecords) { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index a57069d1dc8c..1c6bfcd81bec 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -47,6 +47,7 @@ import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.index.HoodieIndex.IndexType; import com.uber.hoodie.index.bloom.HoodieBloomIndex; import java.io.File; import java.io.IOException; @@ -70,7 +71,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -155,7 +155,7 @@ public void testSimpleInsertAndUpdate() throws Exception { assertNoWriteErrors(statuses); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -166,7 +166,7 @@ public void testSimpleInsertAndUpdate() throws Exception { FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); + metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); @@ -209,9 +209,8 @@ public void testSimpleInsertAndUpdate() throws Exception { assertTrue(dataFilesToRead.findAny().isPresent()); // verify that there is a commit - HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), getConfig(false)); - HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants(); + metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true); + HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants(); assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp(); assertTrue(HoodieTimeline.compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER)); @@ -263,7 +262,7 @@ public void testSimpleInsertUpdateAndDelete() throws Exception { assertNoWriteErrors(statuses); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -274,7 +273,7 @@ public void testSimpleInsertUpdateAndDelete() throws Exception { FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); + metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); @@ -372,7 +371,7 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { client.rollback(newCommitTime); 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()); HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); @@ -408,7 +407,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { assertNoWriteErrors(statuses); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -419,7 +418,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); + metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); @@ -456,7 +455,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { client.rollback(newCommitTime); 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); dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); @@ -482,8 +481,8 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); + hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles); final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get() .getTimestamp(); @@ -500,8 +499,8 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); + hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles); assertFalse(roView.getLatestDataFiles().filter(file -> { if (compactedCommitTime.equals(file.getCommitTime())) { @@ -531,7 +530,7 @@ public void testUpsertPartitioner() throws Exception { assertNoWriteErrors(statuses); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); @@ -542,7 +541,7 @@ public void testUpsertPartitioner() throws Exception { FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, - hoodieTable.getCommitsTimeline().filterCompletedInstants(), allFiles); + metaClient.getCommitsTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); Map parquetFileIdToSize = dataFilesToRead.collect( Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); @@ -591,7 +590,6 @@ public void testUpsertPartitioner() throws Exception { } @Test - @Ignore public void testLogFileCountsAfterCompaction() throws Exception { // insert 100 records HoodieWriteConfig config = getConfig(true); @@ -606,7 +604,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Update all the 100 records HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); newCommitTime = "101"; writeClient.startCommitWithTime(newCommitTime); @@ -614,7 +612,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { List updatedRecords = dataGen.generateUpdates(newCommitTime, records); JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); HoodieIndex index = new HoodieBloomIndex<>(config, jsc); - updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect(); + updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, metaClient).collect(); // Write them to corresponding avro logfiles HoodieTestUtils @@ -623,7 +621,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Verify that all data file has one log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) .collect(Collectors.toList()); @@ -634,14 +632,14 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Do a compaction metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); String commitTime = writeClient.startCompaction(); JavaRDD result = writeClient.compact(commitTime); // Verify that recently written compacted data file has no log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - table = HoodieTable.getHoodieTable(metaClient, config); + table = HoodieTable.getHoodieTable(metaClient, config, jsc); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); assertTrue("Compaction commit should be > than last insert", HoodieTimeline.compareTimestamps( @@ -677,7 +675,7 @@ public void testMetadataValuesAfterInsertUpsertAndCompaction() throws Exception // total time taken for creating files should be greater than 0 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); // Update all the 100 records @@ -690,7 +688,7 @@ public void testMetadataValuesAfterInsertUpsertAndCompaction() throws Exception writeClient.commit(newCommitTime, statuses); // total time taken for upsert all records should be greater than 0 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); // Do a compaction @@ -699,22 +697,168 @@ public void testMetadataValuesAfterInsertUpsertAndCompaction() throws Exception writeClient.commitCompaction(commitTime, statuses); // total time taken for scanning log files should be greater than 0 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); } + @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 records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + JavaRDD 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 records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + JavaRDD statuses = writeClient.insert(recordsRDD, newCommitTime); + // trigger an action + List 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 records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + JavaRDD 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) { return getConfigBuilder(autoCommit).build(); } 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) .withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig( HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false) .withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build()) .forTable("test-trip-table") - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()); } private void assertNoWriteErrors(List statuses) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java index b0f4c41822b2..424980af2451 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java @@ -25,7 +25,10 @@ /** * Within a file group, a slice is a combination of data file written at a commit time and list of - * log files, containing changes to the data file from that commit time + * log files, containing changes to the data file from that commit time. + * File Slice represents a collection of files that when merged will give the snapshot view of the file-group as + * of the latest instant in that file-slice. With asynchronous compaction, the latest file slice will also include + * the set of log files written after the compaction request instant. */ public class FileSlice implements Serializable { @@ -44,17 +47,32 @@ public class FileSlice implements Serializable { */ private HoodieDataFile dataFile; + /** + * Outstanding Compaction request instant time + * For a given file-slice, only one compaction request can be outstanding. + */ + private String outstandingCompactionInstant; + /** * List of appendable log files with real time data - Sorted with greater log version first - * Always empty for copy_on_write storage. */ private final TreeSet logFiles; + /** + * Construct a file-slice from fileId only + * @param fileId + */ + public FileSlice(String fileId) { + this(null, fileId); + } + public FileSlice(String baseCommitTime, String fileId) { this.fileId = fileId; this.baseCommitTime = baseCommitTime; this.dataFile = null; - this.logFiles = new TreeSet<>(HoodieLogFile.getLogVersionComparator()); + this.outstandingCompactionInstant = null; + this.logFiles = new TreeSet<>(HoodieLogFile.getBaseInstantAndLogVersionComparator()); } public void setDataFile(HoodieDataFile dataFile) { @@ -81,13 +99,29 @@ public Optional getDataFile() { return Optional.ofNullable(dataFile); } + public String getOutstandingCompactionInstant() { + return outstandingCompactionInstant; + } + + public void setOutstandingCompactionInstant(String outstandingCompactionInstant) { + this.outstandingCompactionInstant = outstandingCompactionInstant; + } + + public String getBaseInstantForLogAppend() { + if (outstandingCompactionInstant != null) { + return outstandingCompactionInstant; + } + return baseCommitTime; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder("FileSlice {"); sb.append("baseCommitTime=").append(baseCommitTime); + sb.append(", outstandingCompactionInstant=").append(outstandingCompactionInstant); sb.append(", dataFile='").append(dataFile).append('\''); sb.append(", logFiles='").append(logFiles).append('\''); sb.append('}'); return sb.toString(); } -} +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java index f3111a109826..aa448452f1a9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java @@ -18,13 +18,18 @@ package com.uber.hoodie.common.model; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import java.io.Serializable; import java.util.Comparator; import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import java.util.Optional; import java.util.TreeMap; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -32,46 +37,48 @@ */ public class HoodieFileGroup implements Serializable { - public static Comparator getReverseCommitTimeComparator() { - return (o1, o2) -> { - // reverse the order - return o2.compareTo(o1); - }; - } - /** * Partition containing the file group. */ private final String partitionPath; - /** * uniquely identifies the file group */ private final String id; - /** * Slices of files in this group, sorted with greater commit first. */ private final TreeMap fileSlices; - /** * Timeline, based on which all getter work */ private final HoodieTimeline timeline; - /** * The last completed instant, that acts as a high watermark for all getters */ private final Optional lastInstant; - public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) { + private HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline, + Map fileSliceMap) { this.partitionPath = partitionPath; this.id = id; this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator()); + this.fileSlices.putAll(fileSliceMap); this.timeline = timeline; this.lastInstant = timeline.lastInstant(); } + public static Comparator getReverseCommitTimeComparator() { + return getCommitTimeComparator().reversed(); + } + + public static Comparator getCommitTimeComparator() { + return (o1, o2) -> { + // reverse the order + return o1.compareTo(o2); + }; + } + /** * Add a new datafile into the file group */ @@ -211,4 +218,138 @@ public String toString() { sb.append('}'); return sb.toString(); } -} + + /** + * Builder for Hoodie File Group. Ensures file-slices in the file-group satisfy constraints. + * Takes care of merging incomplete file-slices to ensure latest file-slice represents the correct + * rt/ro views in the presence of pending/in-progress compactions + */ + public static class Builder { + + /** + * Slices of files in this group, sorted with greater commit first. + */ + private final TreeMap fileSlices = new TreeMap<>(HoodieFileGroup.getCommitTimeComparator()); + /** + * Partition containing the file group. + */ + private String partitionPath; + /** + * uniquely identifies the file group + */ + private String id; + /** + * Timeline, based on which all getter work + */ + private HoodieTimeline timeline; + + /** + * Helper to merge fake file-slice due to compaction with latest file-slice + * + * @param rawFileSlices File Slices including fake file slices + * @param fakeFileSlice Fake file slice + */ + private static Map adjustFileGroupSlices(TreeMap rawFileSlices, + FileSlice fakeFileSlice) { + final Map mergedFileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator()); + // case where last file-slice is due to compaction request. + // We need to merge the latest 2 file-slices + Entry lastEntry = rawFileSlices.lastEntry(); + Entry penultimateEntry = rawFileSlices.lowerEntry(lastEntry.getKey()); + Preconditions.checkArgument(lastEntry.getValue() == fakeFileSlice, + "Sanity check to ensure the last file-slice is not having the data-file. " + + "Last Entry=" + lastEntry.getValue() + + " Expected=" + fakeFileSlice); + FileSlice merged = mergeFileSlices(lastEntry.getValue(), penultimateEntry.getValue()); + // Create new file-slices for the file-group + rawFileSlices.entrySet().stream().filter( + fileSliceEntry -> { + // All file-slices with base-commit less than that of penultimate entry + return HoodieFileGroup.getReverseCommitTimeComparator().compare(fileSliceEntry.getKey(), + penultimateEntry.getKey()) < 0; + }) + .forEach(fileSliceEntry -> mergedFileSlices.put(fileSliceEntry.getKey(), fileSliceEntry.getValue())); + // Add last Entry to complete + mergedFileSlices.put(merged.getBaseCommitTime(), merged); + return mergedFileSlices; + } + + /** + * Helper to last 2 file-slices so that the last-fileslice base-commit is considered outstanding compaction instant + * + * @param lastSlice Latest File slice for a file-group + * @param penultimateSlice Penultimate file slice for a file-group in commit timeline order + */ + private static FileSlice mergeFileSlices(FileSlice lastSlice, FileSlice penultimateSlice) { + FileSlice merged = new FileSlice(penultimateSlice.getBaseCommitTime(), penultimateSlice.getFileId()); + merged.setOutstandingCompactionInstant(lastSlice.getBaseCommitTime()); + merged.setDataFile(penultimateSlice.getDataFile().get()); + // Add Log files from penultimate and last slices + lastSlice.getLogFiles().forEach(lf -> merged.addLogFile(lf)); + penultimateSlice.getLogFiles().forEach(lf -> merged.addLogFile(lf)); + return merged; + } + + public Builder withPartitionPath(String partitionPath) { + this.partitionPath = partitionPath; + return this; + } + + public Builder withId(String id) { + this.id = id; + return this; + } + + public Builder withTimeline(HoodieTimeline timeline) { + this.timeline = timeline; + return this; + } + + /** + * Add a new datafile into the file group + */ + public Builder withDataFile(HoodieDataFile dataFile) { + if (!fileSlices.containsKey(dataFile.getCommitTime())) { + fileSlices.put(dataFile.getCommitTime(), new FileSlice(dataFile.getCommitTime(), id)); + } + fileSlices.get(dataFile.getCommitTime()).setDataFile(dataFile); + return this; + } + + /** + * Add a new log file into the group + */ + public Builder withLogFile(HoodieLogFile logFile) { + if (!fileSlices.containsKey(logFile.getBaseCommitTime())) { + fileSlices.put(logFile.getBaseCommitTime(), new FileSlice(logFile.getBaseCommitTime(), id)); + } + fileSlices.get(logFile.getBaseCommitTime()).addLogFile(logFile); + return this; + } + + public HoodieFileGroup build() { + Preconditions.checkNotNull(id, "File Id must not be null"); + Preconditions.checkNotNull(timeline, "Timeline must not be null"); + + /** + * We can have at-most last 2 file-slices without base file. This is possible if a compaction request + * has been set for a file-group first time. At this time, the file group may not even have a data file + * (in the case of log files supporting both inserts and updates). In this case, we will merge the last 2 + * file-slices so that we will maintain the constraint that only latest file-slice has compaction request marker + * set. + */ + List fileSlicesWithNoDataFile = + fileSlices.values().stream().filter(f -> !f.getDataFile().isPresent()).collect(Collectors.toList()); + Preconditions.checkArgument(fileSlicesWithNoDataFile.size() <= 2, + "Atmost 2 file-slices with missing data file expected. Got: " + fileSlicesWithNoDataFile); + final Map mergedFileSlices; + if (!fileSlicesWithNoDataFile.isEmpty() && (fileSlices.size() > 1)) { + // last file slice is due to compaction (fake) + mergedFileSlices = adjustFileGroupSlices(fileSlices, Iterables.getLast(fileSlicesWithNoDataFile)); + } else { + mergedFileSlices = fileSlices; + } + return new HoodieFileGroup(partitionPath, id, timeline, mergedFileSlices); + } + } +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java index 7dfaf0bdf9f4..ec2e094d4792 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java @@ -94,10 +94,16 @@ public HoodieLogFile rollOver(FileSystem fs) throws IOException { FSUtils.makeLogFileName(fileId, extension, baseCommitTime, newVersion))); } - public static Comparator getLogVersionComparator() { + public static Comparator getBaseInstantAndLogVersionComparator() { return (o1, o2) -> { - // reverse the order - return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion()); + String baseInstant1 = o1.getBaseCommitTime(); + String baseInstant2 = o2.getBaseCommitTime(); + if (baseInstant1.equals(baseInstant2)) { + // reverse the order by log-version when base-commit is same + return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion()); + } + // reverse the order by base-commits + return new Integer(baseInstant2.compareTo(baseInstant1)); }; } @@ -122,4 +128,4 @@ public int hashCode() { public String toString() { return "HoodieLogFile {" + path + '}'; } -} +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index 9462c301db34..3a89f96d07a6 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -16,12 +16,15 @@ 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.model.HoodieTableType; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.DatasetNotFoundException; +import com.uber.hoodie.exception.HoodieException; import java.io.File; import java.io.IOException; import java.io.Serializable; @@ -198,7 +201,7 @@ public static HoodieTableMetaClient initTableType(Configuration hadoopConf, Stri Properties properties = new Properties(); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName); 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); } return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties); @@ -254,6 +257,53 @@ public static FileStatus[] scanFiles(FileSystem fs, Path metaPath, PathFilter na 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 public boolean equals(Object o) { if (this == o) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java index dd79fff921dc..a1309c73f106 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/AbstractHoodieLogRecordScanner.java @@ -91,13 +91,12 @@ public abstract class AbstractHoodieLogRecordScanner { private AtomicLong totalCorruptBlocks = new AtomicLong(0); // Store the last instant log blocks (needed to implement rollback) private Deque currentInstantLogBlocks = new ArrayDeque<>(); - // Progress private float progress = 0.0f; - public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, - Schema readerSchema, String latestInstantTime, - boolean readBlocksLazily, boolean reverseReader, int bufferSize) { + // TODO (NA) - Change this to a builder, this constructor is too long + public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, + String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index 8978bf91af2d..28d01778907e 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -127,16 +127,17 @@ private List addFilesToView(FileStatus[] statuses) { fileIdSet.addAll(logFiles.keySet()); List fileGroups = new ArrayList<>(); + //TODO: vb: Need to check/fix dataFile corresponding to inprogress/failed async compaction do not show up here fileIdSet.forEach(pair -> { - HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(), - visibleActiveTimeline); + HoodieFileGroup.Builder groupBuilder = new HoodieFileGroup.Builder().withPartitionPath(pair.getKey()) + .withId(pair.getValue()).withTimeline(visibleActiveTimeline); if (dataFiles.containsKey(pair)) { - dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile)); + dataFiles.get(pair).forEach(dataFile -> groupBuilder.withDataFile(dataFile)); } if (logFiles.containsKey(pair)) { - logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile)); + logFiles.get(pair).forEach(logFile -> groupBuilder.withLogFile(logFile)); } - fileGroups.add(group); + fileGroups.add(groupBuilder.build()); }); // add to the cache. @@ -260,4 +261,4 @@ public Stream getAllFileGroups(String partitionPathStr) { "Failed to list data files in partition " + partitionPathStr, e); } } -} +} \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index e837285b1f2e..437a8cfd20e5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -247,6 +247,13 @@ public static String maskWithoutLogVersion(String commitTime, String fileId, 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 diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java index f3fe941000f7..f2f6e4636ce3 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java @@ -24,7 +24,6 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; -import com.uber.hoodie.table.HoodieTable; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.fs.FileSystem; @@ -66,15 +65,13 @@ public static String latestCommit(FileSystem fs, String basePath) { * could be fed into the datasource options. */ public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), - null); - if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) { - return table.getActiveTimeline().getTimelineOfActions( + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); + if (metaClient.getTableType().equals(HoodieTableType.MERGE_ON_READ)) { + return metaClient.getActiveTimeline().getTimelineOfActions( Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION)); } else { - return table.getCommitTimeline().filterCompletedInstants(); + return metaClient.getCommitTimeline().filterCompletedInstants(); } } } diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala index cf0a2c1aa808..cace267a9bcd 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala @@ -21,6 +21,7 @@ package com.uber.hoodie import com.uber.hoodie.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieTableType} import com.uber.hoodie.common.table.HoodieTableMetaClient import com.uber.hoodie.common.util.ParquetUtils +import com.uber.hoodie.config.HoodieWriteConfig import com.uber.hoodie.exception.HoodieException import com.uber.hoodie.table.HoodieTable import org.apache.hadoop.fs.Path @@ -52,8 +53,10 @@ class IncrementalRelation(val sqlContext: SQLContext, if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) { throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets") } - val hoodieTable = HoodieTable.getHoodieTable(metaClient, null) - val commitTimeline = hoodieTable.getCommitTimeline.filterCompletedInstants(); + // TODO : Figure out a valid HoodieWriteConfig + val hoodieTable = HoodieTable.getHoodieTable(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), + sqlContext.sparkContext) + val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants(); if (commitTimeline.empty()) { throw new HoodieException("No instants to incrementally pull") } From c843cc2083b230174dc61d4477be0dcbf6232f6c Mon Sep 17 00:00:00 2001 From: Nishith Agarwal Date: Sun, 20 May 2018 09:06:52 -0700 Subject: [PATCH 2/2] staging changes for rolling stats --- .../com/uber/hoodie/HoodieWriteClient.java | 55 ++- .../uber/hoodie/io/HoodieAppendHandle.java | 24 +- .../hoodie/io/HoodieCommitArchiveLog.java | 3 + .../hoodie/table/HoodieCopyOnWriteTable.java | 16 +- .../hoodie/table/HoodieMergeOnReadTable.java | 315 ++++++++++++------ .../TestHoodieClientOnCopyOnWriteStorage.java | 43 +++ .../hoodie/table/TestMergeOnReadTable.java | 309 ++++++++++++++++- .../src/main/avro/HoodieCommitMetadata.avsc | 12 + .../common/model/HoodieRollingStat.java | 66 ++++ .../model/HoodieRollingStatMetadata.java | 101 ++++++ .../table/timeline/HoodieActiveTimeline.java | 2 +- .../com/uber/hoodie/common/util/FSUtils.java | 25 ++ 12 files changed, 853 insertions(+), 118 deletions(-) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStat.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStatMetadata.java diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 5b506d18125c..e43939bf2d29 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -30,6 +30,8 @@ import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.model.HoodieRollingStat; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -377,8 +379,12 @@ private JavaRDD> combineOnCondition(boolean condition, /** * Save the workload profile in an intermediate file (here re-using commit files) This is useful * when performing rollback for MOR datasets. Only updates are recorded in the workload profile - * metadata since updates to log blocks are unknown across batches Inserts (which are new parquet - * files) are rolled back based on commit time. // TODO : Create a new WorkloadProfile metadata + * metadata since updates to log blocks are unknown across batches. Inserts (which are new parquet + * files and/or new log files) are rolled back based on commit time and whether the partition stats is empty. This is + * done since the FileId's are generated by the IO handler and not the workload profile metadata so there is no + * upfront way to writing out the HoodieRecordLocation (or the fileId) for the incoming inserts. + * // TODO : Create a new WorkloadProfile + * metadata * file instead of using HoodieCommitMetadata */ private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, HoodieTable table, @@ -498,9 +504,7 @@ private boolean commit(String commitTime, JavaRDD writeStatuses, writeStatus.getPartitionPath(), writeStatus.getStat())).collect(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - for (Tuple2 stat : stats) { - metadata.addWriteStat(stat._1(), stat._2()); - } + updateMetadataAndRollingStats(actionType, metadata, stats); // Finalize write final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); @@ -1071,4 +1075,45 @@ private HoodieTable getTableAndInitCtx() { } return table; } + + private void updateMetadataAndRollingStats(String actionType, HoodieCommitMetadata metadata, List> stats) { + // TODO : make sure we cannot rollback / archive last commit file + try { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc); + // 0. All of the rolling stat management is only done by the DELTA commit for MOR and COMMIT for COW other wise + // there may be race conditions + HoodieRollingStatMetadata rollingStatMetadata = new HoodieRollingStatMetadata(actionType); + // 1. Look up previous rolling stat metadata file + // 2. Look up the previous compaction/commit and get the HoodieCommitMetadata from there. + // 3. Now, first read the existing rolling stats and merge with the result of current metadata. + + // Need to do this on every commit (delta or commit) to support COW and MOR. NOTE : For compaction commits, + // this will just add metadata which will never be used + for (Tuple2 stat : stats) { + metadata.addWriteStat(stat._1(), stat._2()); + HoodieRollingStat hoodieRollingStat = new HoodieRollingStat(stat._2().getFileId(), stat._2().getNumWrites(), + stat._2().getNumUpdateWrites(), stat._2().getTotalWriteBytes()); + rollingStatMetadata.addRollingStat(stat._1, hoodieRollingStat); + } + // The last rolling stat should be present in the completed timeline + Optional lastInstant = table.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants() + .lastInstant(); + if (lastInstant.isPresent()) { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(table.getActiveTimeline().getInstantDetails(lastInstant + .get()).get()); + rollingStatMetadata = rollingStatMetadata + .merge(HoodieRollingStatMetadata.fromString(commitMetadata.getExtraMetadata() + .get + (HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY))); + } + + metadata.addMetadata(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, rollingStatMetadata.toJsonString()); + } catch (IOException io) { + throw new HoodieCommitException("unable to save rolling stats"); + } + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index fd482ecebe74..ac946d8f05de 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -89,6 +89,10 @@ public class HoodieAppendHandle extends HoodieIOH private int maxBlockSize = config.getLogFileDataBlockMaxSize(); // Header metadata for a log block private Map header = Maps.newHashMap(); + // Set the base commit time as the current commitTime for new inserts into log files + private String baseCommitTime = commitTime; + // default is false + private boolean isUpsert; public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String fileId, Iterator> recordItr) { @@ -99,10 +103,13 @@ public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTab this.fileId = fileId; this.fileSystemView = hoodieTable.getRTFileSystemView(); this.recordItr = recordItr; + this.isUpsert = true; } public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { this(config, commitTime, hoodieTable, UUID.randomUUID().toString(), null); + // override to false + this.isUpsert = false; } private void init(HoodieRecord record) { @@ -110,8 +117,13 @@ private void init(HoodieRecord record) { this.partitionPath = record.getPartitionPath(); // extract some information from the first record Optional fileSlice = fileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)).findFirst(); - String baseCommitTime = commitTime; + .filter(fileSlice1 -> { + if (fileSlice1.getDataFile().isPresent()) { + return fileSlice1.getDataFile().get().getFileId().equals(fileId); + } else { + return fileSlice1.getLogFiles().findFirst().get().getFileId().equals(fileId); + } + }).findFirst(); if (fileSlice.isPresent()) { baseCommitTime = fileSlice.get().getBaseInstantForLogAppend(); } else { @@ -119,7 +131,6 @@ private void init(HoodieRecord record) { fileSlice = Optional.of(new FileSlice(baseCommitTime, this.fileId)); logger.info("New InsertHandle for partition :" + partitionPath); } - writeStatus.getStat().setPrevCommit(baseCommitTime); writeStatus.setFileId(fileId); writeStatus.setPartitionPath(partitionPath); writeStatus.getStat().setFileId(fileId); @@ -235,9 +246,12 @@ public WriteStatus close() { if (writer != null) { writer.close(); } - writeStatus.getStat().setPrevCommit(commitTime); writeStatus.getStat().setFileId(this.fileId); - writeStatus.getStat().setNumWrites(recordsWritten); + if (!isUpsert) { + writeStatus.getStat().setNumWrites(recordsWritten); + } else { + writeStatus.getStat().setNumUpdateWrites(recordsWritten); + } writeStatus.getStat().setNumDeletes(recordsDeleted); writeStatus.getStat().setTotalWriteBytes(estimatedNumberOfBytesWritten); writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index bf78467ffe38..90258adbcf01 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -27,6 +27,7 @@ import com.uber.hoodie.common.model.ActionType; import com.uber.hoodie.common.model.HoodieArchivedLogFile; import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.log.HoodieLogFormat; @@ -253,6 +254,8 @@ private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter( mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData = mapper .convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class); + // Do not archive Rolling Stats + avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, null); return avroMetaData; } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index 9e6850bed8bd..131c26cd2102 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -27,6 +27,7 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -640,13 +641,18 @@ class UpsertPartitioner extends Partitioner { * Remembers what type each bucket is for later. */ private HashMap bucketInfoMap; + /** + * Rolling stats for files + */ + protected HoodieRollingStatMetadata rollingStatMetadata; + protected long averageRecordSize; UpsertPartitioner(WorkloadProfile profile) { updateLocationToBucket = new HashMap<>(); partitionPathToInsertBuckets = new HashMap<>(); bucketInfoMap = new HashMap<>(); globalStat = profile.getGlobalStat(); - + rollingStatMetadata = getRollingStats(); assignUpdates(profile); assignInserts(profile); @@ -679,7 +685,7 @@ private int addUpdateBucket(String fileLoc) { private void assignInserts(WorkloadProfile profile) { // for new inserts, compute buckets depending on how many records we have for each partition Set partitionPaths = profile.getPartitionPaths(); - long averageRecordSize = averageBytesPerRecord(); + averageRecordSize = averageBytesPerRecord(); logger.info("AvgRecordSize => " + averageRecordSize); for (String partitionPath : partitionPaths) { WorkloadStat pStat = profile.getWorkloadStat(partitionPath); @@ -790,7 +796,7 @@ protected List getSmallFiles(String partitionPath) { * Obtains the average record size based on records written during last commit. Used for * estimating how many records pack into one file. */ - private long averageBytesPerRecord() { + protected long averageBytesPerRecord() { long avgSize = 0L; HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getCommitTimeline() .filterCompletedInstants(); @@ -850,4 +856,8 @@ public int getPartition(Object key) { } } } + + protected HoodieRollingStatMetadata getRollingStats() { + return null; + } } \ No newline at end of file diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 395c1394c6ee..f9de73b98710 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -26,10 +26,11 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.model.HoodieRollingStat; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.table.HoodieTimeline; 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.HoodieCommandBlockTypeEnum; import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType; @@ -38,13 +39,11 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCompactionException; +import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.exception.HoodieUpsertException; -<<<<<<< 23d53763c4db744fe936e2e609fe6abaf7bd9a82 -import com.uber.hoodie.index.HoodieIndex; -======= import com.uber.hoodie.func.MergeOnReadLazyInsertIterable; ->>>>>>> Adding ability for inserts to be written to log files +import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.io.HoodieAppendHandle; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; import java.io.IOException; @@ -100,8 +99,8 @@ public Partitioner getUpsertPartitioner(WorkloadProfile profile) { public Iterator> handleUpdate(String commitTime, String fileId, Iterator> recordItr) throws IOException { logger.info("Merging updates for commit " + commitTime + " for file " + fileId); - - if (mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { + // Enabled small file handling for log files + if (!index.canIndexLogFiles() && mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { logger.info( "Small file corrections for updates for commit " + commitTime + " for file " + fileId); return super.handleUpdate(commitTime, fileId, recordItr); @@ -158,7 +157,7 @@ public JavaRDD compact(JavaSparkContext jsc, String compactionCommi public List rollback(JavaSparkContext jsc, List commits) throws IOException { - //At the moment, MOR table type does not support nested rollbacks + // At the moment, MOR table type does not support nested rollbacks if (commits.size() > 1) { throw new UnsupportedOperationException("Nested Rollbacks are not supported"); } @@ -173,8 +172,6 @@ public List rollback(JavaSparkContext jsc, List comm .filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight); logger.info("Unpublished " + commits); Long startTime = System.currentTimeMillis(); - // TODO (NA) : remove this once HoodieIndex is a member of HoodieTable - HoodieIndex hoodieIndex = HoodieIndex.createIndex(config, jsc); List allRollbackStats = jsc.parallelize(FSUtils .getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) @@ -194,6 +191,29 @@ public List rollback(JavaSparkContext jsc, List comm throw new UncheckedIOException("Failed to rollback for commit " + commit, io); } case HoodieTimeline.DELTA_COMMIT_ACTION: + // -------------------------------------------------------------------------------------------------- + // (A) The following cases are possible if index.canIndexLogFiles and/or index.isGlobal + // -------------------------------------------------------------------------------------------------- + // (A.1) Failed first commit - Inserts were written to log files and HoodieWriteStat has no entries. In + // this scenario we would want to delete these log files. + // (A.2) Failed recurring commit - Inserts/Updates written to log files. In this scenario, + // HoodieWriteStat will have the baseCommitTime for the first log file written, add rollback blocks. + // (A.3) Rollback triggered for first commit - Inserts were written to the log files but the commit is + // being reverted. In this scenario, HoodieWriteStat will be `null` for the attribute prevCommitTime and + // and hence will end up deleting these log files. This is done so there are no orphan log files + // lying around. + // (A.4) Rollback triggered for recurring commits - Inserts/Updates are being rolled back, the actions + // taken in this scenario is a combination of (A.2) and (A.3) + // --------------------------------------------------------------------------------------------------- + // (B) The following cases are possible if !index.canIndexLogFiles and/or !index.isGlobal + // --------------------------------------------------------------------------------------------------- + // (B.1) Failed first commit - Inserts were written to parquet files and HoodieWriteStat has no entries. + // In this scenario, we delete all the parquet files written for the failed commit. + // (B.2) Failed recurring commits - Inserts were written to parquet files and updates to log files. In + // this scenario, perform (A.1) and for updates written to log files, write rollback blocks. + // (B.3) Rollback triggered for first commit - Same as (B.1) + // (B.4) Rollback triggered for recurring commits - Same as (B.2) plus we need to delete the log files + // as well if the base parquet file gets deleted. try { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( metaClient.getCommitTimeline().getInstantDetails( @@ -203,83 +223,15 @@ public List rollback(JavaSparkContext jsc, List comm // read commit file and (either append delete blocks or delete file) final Map filesToDeletedStatus = new HashMap<>(); Map filesToNumBlocksRollback = new HashMap<>(); - - // In case all data was inserts and the commit failed, there is no partition stats + // In case all data was inserts and the commit failed, there is no partition stats since no records + // were tagged in the workload profile. if (commitMetadata.getPartitionToWriteStats().size() == 0) { super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, Arrays.asList(commit)); } - - // append rollback blocks for updates + // Append rollback blocks for updates if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { - // This needs to be done since GlobalIndex at the moment does not store the latest commit time - // Also, the baseCommitTime changes if compaction has been requested so we need to do this anyways - Map fileIdToLatestCommitTimeMap = - hoodieIndex.isGlobal() ? this.getRTFileSystemView().getLatestFileSlices(partitionPath) - .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantForLogAppend)) : null; - commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() - .filter(wStat -> { - if (wStat != null - && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT - && wStat.getPrevCommit() != 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, Arrays.asList(commit)); - } catch (IOException io) { - throw new UncheckedIOException(io); - } - return false; - }).forEach(wStat -> { - HoodieLogFormat.Writer writer = null; - String baseCommitTime = wStat.getPrevCommit(); - if (hoodieIndex.isGlobal()) { - baseCommitTime = fileIdToLatestCommitTimeMap.get(wStat.getFileId()); - } - 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 - // getBaseCommitTimeForAppendLog() - writer = HoodieLogFormat.newWriterBuilder().onParentPath( - new Path(this.getMetaClient().getBasePath(), partitionPath)) - .withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime) - .withFs(this.metaClient.getFs()) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); - Long numRollbackBlocks = 0L; - // generate metadata - Map header = - Maps.newHashMap(); - header.put(HeaderMetadataType.INSTANT_TIME, - metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); - header.put(HeaderMetadataType.TARGET_INSTANT_TIME, - commit); - header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String - .valueOf( - HoodieCommandBlockTypeEnum - .ROLLBACK_PREVIOUS_BLOCK - .ordinal())); - // if update belongs to an existing log file - writer = writer.appendBlock(new HoodieCommandBlock(header)); - numRollbackBlocks++; - filesToNumBlocksRollback.put(this.getMetaClient().getFs() - .getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks); - } catch (IOException | InterruptedException io) { - throw new HoodieRollbackException( - "Failed to rollback for commit " + commit, io); - } finally { - try { - writer.close(); - } catch (IOException io) { - throw new UncheckedIOException(io); - } - } - }); - hoodieRollbackStats = HoodieRollbackStat.newBuilder() - .withPartitionPath(partitionPath) - .withDeletedFileResults(filesToDeletedStatus) - .withRollbackBlockAppendResults(filesToNumBlocksRollback).build(); + hoodieRollbackStats = rollback(index, partitionPath, commit, commitMetadata, filesToDeletedStatus, + filesToNumBlocksRollback); } break; } catch (IOException io) { @@ -308,6 +260,26 @@ public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) return Optional.empty(); } + @Override + protected HoodieRollingStatMetadata getRollingStats() { + try { + Optional lastInstant = this.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants() + .lastInstant(); + if (lastInstant.isPresent()) { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(this.getActiveTimeline().getInstantDetails + (lastInstant.get()).get()); + HoodieRollingStatMetadata rollingStatMetadata = HoodieRollingStatMetadata + .fromString(commitMetadata.getExtraMetadata().get + (HoodieRollingStatMetadata + .ROLLING_STAT_METADATA_KEY)); + return rollingStatMetadata; + } + return null; + } catch (IOException e) { + throw new HoodieException(); + } + } + /** * UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet * files to larger ones without the need for an index in the logFile. @@ -327,31 +299,60 @@ protected List getSmallFiles(String partitionPath) { // Init here since this class (and member variables) might not have been initialized HoodieTimeline commitTimeline = getCompletedCommitTimeline(); + // Find out all eligible small file slices if (!commitTimeline.empty()) { HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); // find smallest file in partition and append to it + List allSmallFileSlices = new ArrayList<>(); + // If we cannot index log files, then we choose the smallest parquet file in the partition and add inserts to + // it. Doing this overtime for a partition, we ensure that we handle small file issues + if (!index.canIndexLogFiles()) { + // TODO : choose last N small files since there can be multiple small files written to a single partitions + // in a single batch + Optional smallFile = getRTFileSystemView() + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter( + fileSlice -> fileSlice.getLogFiles().count() < 1 + && fileSlice.getDataFile().get().getFileSize() < config + .getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) -> + left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() + ? -1 : 1).findFirst(); + if (smallFile.isPresent()) { + allSmallFileSlices.add(smallFile.get()); + } + } else { + // If we can index log files, we can add more inserts to log files. + List allFiles = getRTFileSystemView() + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()) + .collect(Collectors.toList()); + for (FileSlice fileSlice : allFiles) { + if (isSmallFile(partitionPath, fileSlice)) { + allSmallFileSlices.add(fileSlice); + } + } + } - // TODO - check if index.isglobal then small files are log files too - Optional smallFileSlice = getRTFileSystemView() - .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter( - fileSlice -> fileSlice.getLogFiles().count() < 1 - && fileSlice.getDataFile().get().getFileSize() < config - .getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) -> - left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() - ? -1 : 1).findFirst(); - - if (smallFileSlice.isPresent()) { - String filename = smallFileSlice.get().getDataFile().get().getFileName(); + // Create SmallFiles from the eligible file slices + for (FileSlice smallFileSlice : allSmallFileSlices) { SmallFile sf = new SmallFile(); - sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), - FSUtils.getFileId(filename)); - sf.sizeBytes = smallFileSlice.get().getDataFile().get().getFileSize(); - smallFileLocations.add(sf); - // Update the global small files list - smallFiles.add(sf); + if (smallFileSlice.getDataFile().isPresent()) { + // TODO : Move logic of file name, file id, base commit time handling inside file slice + String filename = smallFileSlice.getDataFile().get().getFileName(); + sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename)); + sf.sizeBytes = getTotalFileSize(partitionPath, smallFileSlice); + smallFileLocations.add(sf); + // Update the global small files list + smallFiles.add(sf); + } else { + HoodieLogFile logFile = smallFileSlice.getLogFiles().findFirst().get(); + sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()), + FSUtils.getFileIdFromLogPath(logFile.getPath())); + sf.sizeBytes = getTotalFileSize(partitionPath, smallFileSlice); + smallFileLocations.add(sf); + // Update the global small files list + smallFiles.add(sf); + } } } - return smallFileLocations; } @@ -360,6 +361,118 @@ public List getSmallFileIds() { .map(smallFile -> ((SmallFile) smallFile).location.getFileId()) .collect(Collectors.toList()); } + + private long getTotalFileSize(String partitionPath, FileSlice fileSlice) { + if (rollingStatMetadata != null) { + Map partitionRollingStats = rollingStatMetadata.getPartitionToRollingStats().get + (partitionPath); + if (partitionRollingStats != null) { + HoodieRollingStat rollingStatForFile = partitionRollingStats.get(fileSlice.getFileId()); + if (rollingStatForFile != null) { + long inserts = rollingStatForFile.getInserts(); + long totalSize = averageRecordSize * inserts; + return totalSize; + } + } + } + // In case Rolling Stats is not present, fall back to sizing log files based on heuristics + if (!fileSlice.getDataFile().isPresent()) { + return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList())); + } else { + return fileSlice.getDataFile().get().getFileSize() + convertLogFilesSizeToExpectedParquetSize(fileSlice + .getLogFiles().collect(Collectors.toList())); + } + } + + private boolean isSmallFile(String partitionPath, FileSlice fileSlice) { + long totalSize = getTotalFileSize(partitionPath, fileSlice); + if (totalSize < config.getParquetMaxFileSize()) { + return true; + } + return false; + } + + // TODO (NA) : Make this static part of utility + protected long convertLogFilesSizeToExpectedParquetSize(List hoodieLogFiles) { + long totalSizeOfLogFiles = hoodieLogFiles.stream().map(hoodieLogFile -> hoodieLogFile.getFileSize().get()) + .reduce((a, b) -> (a + b)).orElse(0L); + // Here we assume that if there is no base parquet file, all log files contain only inserts. + // We can then just get the parquet equivalent size of these log files, compare that with + // {@link config.getParquetMaxFileSize()} and decide if there is scope to insert more rows + long logFilesEquivalentParquetFileSize = (long) (totalSizeOfLogFiles * config + .getLogFileToParquetCompressionRatio()); + return logFilesEquivalentParquetFileSize; + } } + private Map generateHeader(String commit) { + // generate metadata + Map header = Maps.newHashMap(); + header.put(HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); + header.put(HeaderMetadataType.TARGET_INSTANT_TIME, commit); + header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK + .ordinal())); + return header; + } + + private HoodieRollbackStat rollback(HoodieIndex hoodieIndex, String partitionPath, String commit, + HoodieCommitMetadata commitMetadata, final Map filesToDeletedStatus, + Map filesToNumBlocksRollback) { + // The following needs to be done since GlobalIndex at the moment does not store the latest commit time. + // Also, wStat.getPrevCommit() might not give the right commit time in the following + // scenario : If a compaction was scheduled, the new commitTime associated with the requested compaction will be + // used to write the new log files. In this case, the commit time for the log file is the compaction requested time. + Map fileIdToBaseCommitTimeForLogMap = + hoodieIndex.isGlobal() ? this.getRTFileSystemView().getLatestFileSlices(partitionPath) + .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantForLogAppend)) : null; + commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() + .filter(wStat -> { + // Filter out stats without prevCommit since they are all inserts + if (wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != 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, Arrays.asList(commit)); + } catch (IOException io) { + throw new UncheckedIOException(io); + } + return false; + }).forEach(wStat -> { + HoodieLogFormat.Writer writer = null; + String baseCommitTime = wStat.getPrevCommit(); + if (hoodieIndex.isGlobal()) { + baseCommitTime = fileIdToBaseCommitTimeForLogMap.get(wStat.getFileId()); + } + try { + writer = HoodieLogFormat.newWriterBuilder().onParentPath( + new Path(this.getMetaClient().getBasePath(), partitionPath)) + .withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime) + .withFs(this.metaClient.getFs()) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); + Long numRollbackBlocks = 0L; + // generate metadata + Map header = generateHeader(commit); + // if update belongs to an existing log file + writer = writer.appendBlock(new HoodieCommandBlock(header)); + numRollbackBlocks++; + filesToNumBlocksRollback.put(this.getMetaClient().getFs() + .getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks); + } catch (IOException | InterruptedException io) { + throw new HoodieRollbackException( + "Failed to rollback for commit " + commit, io); + } finally { + try { + writer.close(); + } catch (IOException io) { + throw new UncheckedIOException(io); + } + } + }); + return HoodieRollbackStat.newBuilder() + .withPartitionPath(partitionPath) + .withDeletedFileResults(filesToDeletedStatus) + .withRollbackBlockAppendResults(filesToNumBlocksRollback).build(); + } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index 9fbd0871a41d..2beb86bacb89 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -28,6 +28,8 @@ import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRollingStat; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; @@ -46,6 +48,7 @@ import java.util.Collection; import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -54,6 +57,7 @@ import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; +import org.junit.Assert; import org.junit.Test; import scala.Option; @@ -581,6 +585,45 @@ public void testCommitWritesRelativePaths() throws Exception { } } + /** + * Test to ensure commit metadata points to valid files + */ + @Test + public void testRollingStatsInMetadata() throws Exception { + + HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + + String commitTime = "000"; + client.startCommitWithTime(commitTime); + + List records = dataGen.generateInserts(commitTime, 200); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + JavaRDD result = client.bulkInsert(writeRecords, commitTime); + + assertTrue("Commit should succeed", client.commit(commitTime, result)); + assertTrue("After explicit commit, commit file should be created", + HoodieTestUtils.doesCommitExist(basePath, commitTime)); + + // Read from commit file + String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime); + FileInputStream inputStream = new FileInputStream(filename); + String everything = IOUtils.toString(inputStream); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString()); + HoodieRollingStatMetadata rollingStatMetadata = HoodieRollingStatMetadata.fromString(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY)); + int inserts = 0; + for (Map.Entry> pstat : rollingStatMetadata.getPartitionToRollingStats().entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + inserts += stat.getValue().getInserts(); + } + } + Assert.assertEquals(inserts, 200); + } + /** * Build Hoodie Write Config for small data file sizes */ diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 1c6bfcd81bec..bf1feb7d65dc 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -31,26 +31,33 @@ import com.uber.hoodie.common.TestRawTripPayload.MetadataMergeWriteStatus; import com.uber.hoodie.common.minicluster.HdfsTestService; import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.model.HoodieCommitMetadata; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieRollingStat; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.TableFileSystemView; +import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieMemoryConfig; import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.index.HoodieIndex.IndexType; import com.uber.hoodie.index.bloom.HoodieBloomIndex; import java.io.File; +import java.io.FileInputStream; import java.io.IOException; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -58,6 +65,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.avro.generic.GenericRecord; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -741,7 +749,85 @@ public void testSimpleInsertsGeneratedIntoLogFiles() throws Exception { } @Test - public void testInsertsGeneratedIntoLogFilesRollback() throws Exception { + public void testInsertsIntoLogFilesWithSmallFileHandling() 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 records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + JavaRDD 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); + + newCommitTime = "101"; + writeClient.startCommitWithTime(newCommitTime); + + records = dataGen.generateInserts(newCommitTime, 100); + recordsRDD = jsc.parallelize(records, 1); + statuses = writeClient.insert(recordsRDD, newCommitTime); + writeClient.commit(newCommitTime, statuses); + + table = HoodieTable + .getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, + jsc); + tableRTFileSystemView = table.getRTFileSystemView(); + + long numLogFiles1 = 0; + List fileIds = new ArrayList<>(); + 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); + numLogFiles1 += tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> + fileSlice.getLogFiles().count() > 0).count(); + fileIds.addAll(tableRTFileSystemView.getLatestFileSlices(partitionPath).map(fileSlice -> + fileSlice.getLogFiles().findFirst().get().getFileId()).collect(Collectors.toList())); + } + + // The inserts should goto the same log files and no new log files should be created + Assert.assertEquals(numLogFiles, numLogFiles1); + + // Compact the log files to generate + String compactionTime = writeClient.startCompaction(); + statuses = writeClient.compact(compactionTime); + writeClient.commitCompaction(compactionTime, statuses); + + // New inserts should still goto the same file-ids + records = dataGen.generateInserts(newCommitTime, 100); + recordsRDD = jsc.parallelize(records, 1); + newCommitTime = "103"; + writeClient.startCommitWithTime(newCommitTime); + statuses = writeClient.insert(recordsRDD, newCommitTime); + writeClient.commit(newCommitTime, statuses); + List newFileIds = statuses.map(status -> status.getFileId()).collect(); + Assert.assertTrue(fileIds.containsAll(newFileIds)); + + } + + @Test + public void testFirstInsertRollback() throws Exception { // insert 100 records // Setting IndexType to be InMemory to simulate Global Index nature HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build(); @@ -765,10 +851,117 @@ public void testInsertsGeneratedIntoLogFilesRollback() throws Exception { // rollback a failed commit boolean rollback = writeClient.rollback(newCommitTime); Assert.assertTrue(rollback); + + final HoodieTableMetaClient metaClient1 = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient1, 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.assertEquals(0, numLogFiles); + + // Insert another 100 records, should be written to the same log files newCommitTime = "101"; writeClient.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 100); + recordsRDD = jsc.parallelize(records, 1); + statuses = writeClient.upsert(recordsRDD, newCommitTime); + writeClient.commit(newCommitTime, statuses); + + // rollback a successful commit + rollback = writeClient.rollback(newCommitTime); + Assert.assertTrue(rollback); + + final HoodieTableMetaClient metaClient2 = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + table = HoodieTable.getHoodieTable(metaClient2, config, jsc); + tableRTFileSystemView = table.getRTFileSystemView(); + 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.assertEquals(0, numLogFiles); + + } + + @Test + public void testRecurringInsertOrUpsertRollback() throws Exception { + // TODO (NA): Remove the HoodieCompactedScanner code once RealtimeInputFormat is fixed to read fileslices without + // base parquet files // 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 records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + JavaRDD statuses = writeClient.insert(recordsRDD, newCommitTime); + // trigger an action + List writeStatuses = statuses.collect(); + writeClient.commit(newCommitTime, statuses); + + // 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); + + // Insert another 100 records, should be written to the same log files due to small file sizing + newCommitTime = "101"; + writeClient.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 100); + recordsRDD = jsc.parallelize(records, 1); + statuses = writeClient.upsert(recordsRDD, newCommitTime); + // trigger an action + statuses.collect(); + + // rollback a failed commit + boolean rollback = writeClient.rollback(newCommitTime); + Assert.assertTrue(rollback); + + final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable + .getHoodieTable(metaClient, config, + jsc); + String latestCommit = table.metaClient.getActiveTimeline().getAllCommitsTimeline().lastInstant().get() + .getTimestamp(); + long totalRecords = 0; + for (String partitionPath : dataGen.getPartitionPaths()) { + totalRecords += table.getRTFileSystemView().getAllFileSlices(partitionPath).map(fileSlice -> { + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(metaClient.getFs(), metaClient + .getBasePath(), fileSlice.getLogFiles().map(hoodieLogFile -> hoodieLogFile.getPath() + .toString()) + .collect(Collectors.toList()), HoodieTestDataGenerator + .avroSchema, + latestCommit, + HoodieMemoryConfig + .DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, Boolean.valueOf(HoodieCompactionConfig + .DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED), false, HoodieMemoryConfig + .DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE, "/tmp"); + return scanner.getRecords().size(); + }).reduce((a,b) -> a + b).get(); + } + + Assert.assertEquals(totalRecords, 100); + // Insert another 100 records + newCommitTime = "102"; + writeClient.startCommitWithTime(newCommitTime); records = dataGen.generateInserts(newCommitTime, 100); recordsRDD = jsc.parallelize(records, 1); statuses = writeClient.insert(recordsRDD, newCommitTime); @@ -776,11 +969,43 @@ public void testInsertsGeneratedIntoLogFilesRollback() throws Exception { // rollback a successful commit writeClient.rollback(newCommitTime); - final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); - HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); + final HoodieTableMetaClient metaClient1 = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + table = HoodieTable.getHoodieTable(metaClient1, config, jsc); TableFileSystemView.RealtimeView tableRTFileSystemView = table.getRTFileSystemView(); long numLogFiles = 0; + totalRecords = 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(); + totalRecords += table.getRTFileSystemView().getAllFileSlices(partitionPath).map(fileSlice -> { + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(metaClient.getFs(), metaClient + .getBasePath(), fileSlice.getLogFiles().map(hoodieLogFile -> hoodieLogFile.getPath() + .toString()) + .collect(Collectors.toList()), HoodieTestDataGenerator + .avroSchema, + latestCommit, + HoodieMemoryConfig + .DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, Boolean.valueOf(HoodieCompactionConfig + .DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED), false, HoodieMemoryConfig + .DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE, "/tmp"); + return scanner.getRecords().size(); + }).reduce((a,b) -> a + b).get(); + } + Assert.assertTrue(numLogFiles > 0); + Assert.assertEquals(totalRecords, 100); + + // Rollback the first commit, this should delete the first log files that were generated + writeClient.rollback("100"); + + final HoodieTableMetaClient metaClient2 = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + table = HoodieTable.getHoodieTable(metaClient2, config, jsc); + tableRTFileSystemView = table.getRTFileSystemView(); + numLogFiles = 0; for (String partitionPath : dataGen.getPartitionPaths()) { Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice -> fileSlice.getDataFile().isPresent()).count() == 0); @@ -842,6 +1067,84 @@ public void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction() throws Exc } } + /** + * Test to ensure rolling stats are correctly written to metadata file + */ + @Test + public void testRollingStatsInMetadata() throws Exception { + + HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + + String commitTime = "000"; + client.startCommitWithTime(commitTime); + + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + List records = dataGen.generateInserts(commitTime, 200); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + JavaRDD statuses = client.insert(writeRecords, commitTime); + assertTrue("Commit should succeed", client.commit(commitTime, statuses)); + + // Read from commit file + table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table + .getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get()); + HoodieRollingStatMetadata rollingStatMetadata = HoodieRollingStatMetadata.fromString(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY)); + int inserts = 0; + for (Map.Entry> pstat : rollingStatMetadata.getPartitionToRollingStats().entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + inserts += stat.getValue().getInserts(); + } + } + Assert.assertEquals(inserts, 200); + + commitTime = "001"; + client.startCommitWithTime(commitTime); + records = dataGen.generateUpdates(commitTime, records); + writeRecords = jsc.parallelize(records, 1); + statuses = client.upsert(writeRecords, commitTime); + assertTrue("Commit should succeed", client.commit(commitTime, statuses)); + + // Read from commit file + table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table + .getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get()); + rollingStatMetadata = HoodieRollingStatMetadata.fromString(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY)); + inserts = 0; int upserts = 0; + for (Map.Entry> pstat : rollingStatMetadata.getPartitionToRollingStats().entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + inserts += stat.getValue().getInserts(); + upserts += stat.getValue().getUpserts(); + } + } + + Assert.assertEquals(inserts, 200); + Assert.assertEquals(upserts, 200); + + client.rollback(commitTime); + + // Read from commit file + table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); + metadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(table + .getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get()); + rollingStatMetadata = HoodieRollingStatMetadata.fromString(metadata.getExtraMetadata() + .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY)); + inserts = 0; upserts = 0; + for (Map.Entry> pstat : rollingStatMetadata.getPartitionToRollingStats().entrySet()) { + for (Map.Entry stat : pstat.getValue().entrySet()) { + inserts += stat.getValue().getInserts(); + upserts += stat.getValue().getUpserts(); + } + } + Assert.assertEquals(inserts, 200); + Assert.assertEquals(upserts, 0); + } + private HoodieWriteConfig getConfig(Boolean autoCommit) { return getConfigBuilder(autoCommit).build(); diff --git a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc index 832b5fc9ccf7..90d240b107cb 100644 --- a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc +++ b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc @@ -60,6 +60,18 @@ { "name":"totalUpdatedRecordsCompacted", "type":["null","long"] + }, + { + "name":"totalLogBlocks", + "type":["null","long"] + }, + { + "name":"totalCorruptLogBlock", + "type":["null","long"] + }, + { + "name":"totalRollbackBlocks", + "type":["null","long"] } ] } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStat.java new file mode 100644 index 000000000000..370a49bf8675 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStat.java @@ -0,0 +1,66 @@ +package com.uber.hoodie.common.model; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import java.io.Serializable; +import javax.annotation.Nullable; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class HoodieRollingStat implements Serializable { + + private String fileId; + private long inserts; + private long upserts; + // TODO + @Nullable + private long totalInputWriteBytesToDisk; + @Nullable + private long totalInputWriteBytesOnDisk; + + public HoodieRollingStat() { + // called by jackson json lib + } + + public HoodieRollingStat(String fileId, long inserts, long upserts, long totalInputWriteBytesOnDisk) { + this.fileId = fileId; + this.inserts = inserts; + this.upserts = upserts; + this.totalInputWriteBytesOnDisk = totalInputWriteBytesOnDisk; + } + + public String getFileId() { + return fileId; + } + + public void setFileId(String fileId) { + this.fileId = fileId; + } + + public long getInserts() { + return inserts; + } + + public void setInserts(long inserts) { + this.inserts = inserts; + } + + public long getUpserts() { + return upserts; + } + + public void setUpserts(long upserts) { + this.upserts = upserts; + } + public long updateInserts(long inserts) { + this.inserts += inserts; + return this.inserts; + } + + public long updateUpserts(long upserts) { + this.upserts += upserts; + return this.upserts; + } + + public long getTotalInputWriteBytesOnDisk() { + return totalInputWriteBytesOnDisk; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStatMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStatMetadata.java new file mode 100644 index 000000000000..ce899689b9b7 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRollingStatMetadata.java @@ -0,0 +1,101 @@ +package com.uber.hoodie.common.model; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.PropertyAccessor; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.Charset; +import java.util.HashMap; +import java.util.Map; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +public class HoodieRollingStatMetadata implements Serializable { + + private static volatile Logger log = LogManager.getLogger(HoodieRollingStatMetadata.class); + protected Map> partitionToRollingStats; + private String actionType = "DUMMY_ACTION"; + public static final String ROLLING_STAT_METADATA_KEY = "ROLLING_STAT"; + + public void addRollingStat(String partitionPath, HoodieRollingStat stat) { + if (!partitionToRollingStats.containsKey(partitionPath)) { + partitionToRollingStats.put(partitionPath, new RollingStatsHashMap<>()); + } + partitionToRollingStats.get(partitionPath).put(stat.getFileId(), stat); + } + + public HoodieRollingStatMetadata() { + partitionToRollingStats = new HashMap<>(); + } + + public HoodieRollingStatMetadata(String actionType) { + this(); + this.actionType = actionType; + } + + class RollingStatsHashMap extends HashMap { + + @Override + public V put(K key, V value) { + V v = this.get(key); + if (v == null) { + super.put(key, value); + } else if (v instanceof HoodieRollingStat) { + if (HoodieActiveTimeline.COMMIT_ACTION.equals(actionType)) { + return super.remove(key); + } else { + long inserts = ((HoodieRollingStat) v).getInserts(); + long upserts = ((HoodieRollingStat) v).getUpserts(); + ((HoodieRollingStat) value).updateInserts(inserts); + ((HoodieRollingStat) value).updateUpserts(upserts); + super.put(key, value); + } + } + return value; + } + } + + public static HoodieRollingStatMetadata fromBytes(byte[] bytes) throws IOException { + return fromString(new String(bytes, Charset.forName("utf-8"))); + } + + public static HoodieRollingStatMetadata fromString(String jsonStr) throws IOException { + return getObjectMapper().readValue(jsonStr, HoodieRollingStatMetadata.class); + } + + public String toJsonString() throws IOException { + if (partitionToRollingStats.containsKey(null)) { + log.info("partition path is null for " + partitionToRollingStats.get(null)); + partitionToRollingStats.remove(null); + } + return getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); + } + + private static ObjectMapper getObjectMapper() { + ObjectMapper mapper = new ObjectMapper(); + mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); + return mapper; + } + + public HoodieRollingStatMetadata merge(HoodieRollingStatMetadata rollingStatMetadata) { + for (Map.Entry> stat : rollingStatMetadata.partitionToRollingStats + .entrySet()) { + for (Map.Entry innerStat : stat.getValue().entrySet()) { + this.addRollingStat(stat.getKey(), innerStat.getValue()); + } + } + return this; + } + + public Map> getPartitionToRollingStats() { + return partitionToRollingStats; + } + + public String getActionType() { + return actionType; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index a8d35411e155..829b0ee72b1b 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -269,7 +269,7 @@ public void saveToInflight(HoodieInstant instant, Optional content) { createFileInMetaPath(instant.getFileName(), content); } - protected void createFileInMetaPath(String filename, Optional content) { + public void createFileInMetaPath(String filename, Optional content) { Path fullPath = new Path(metaClient.getMetaPath(), filename); try { if (!content.isPresent()) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index 437a8cfd20e5..24ad0901dae1 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -19,6 +19,7 @@ import com.google.common.base.Preconditions; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodiePartitionMetadata; +import com.uber.hoodie.common.model.HoodieRollingStatMetadata; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.InvalidHoodiePathException; @@ -33,7 +34,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Stream; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; @@ -386,4 +389,26 @@ public static void createPathIfNotExists(FileSystem fs, Path partitionPath) thro public static Long getSizeInMB(long sizeInBytes) { return sizeInBytes / (1024 * 1024); } + +// public static Optional getLastRollingStat(String basePath, FileSystem fs) throws +// IOException { +// List fileStatuses = Arrays.asList(fs.listStatus(new Path(basePath + "/.hoodie/rolling/"))); +// Arrays.asList(fileStatuses).sort((FileStatus f1, FileStatus f2) -> { +// return f1.getPath().getName() > f2.getPath().getName(); +// }); +// FileStatus fileStatus = fileStatuses.stream().findFirst().get(); +// Optional rollingStats = readDataFromPath(fs, fileStatus.getPath()); +// if (rollingStats.isPresent()) { +// return Optional.of(HoodieRollingStatMetadata.fromBytes(rollingStats.get())); +// } +// return Optional.empty(); +// } + +// public static Optional readDataFromPath(FileSystem fs, Path detailPath) { +// try (FSDataInputStream is = fs.open(detailPath)) { +// return Optional.of(IOUtils.toByteArray(is)); +// } catch (IOException e) { +// throw new HoodieIOException("Could not read commit details from " + detailPath, e); +// } +// } }