From 788e4f2d2eeeeee685b9f5d2e09b56be230e34d4 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Tue, 20 Mar 2018 16:29:20 -0700 Subject: [PATCH] CodeStyle formatting to conform to basic Checkstyle rules. The code-style rules follow google style with some changes: 1. Increase line length from 100 to 120 2. Disable JavaDoc related checkstyles as this needs more manual work. Both source and test code are checked for code-style --- docs/dev_setup.md | 9 +- .../com/uber/hoodie/cli/HoodiePrompt.java | 6 +- .../uber/hoodie/cli/HoodieSplashScreen.java | 26 +- .../main/java/com/uber/hoodie/cli/Main.java | 3 +- .../cli/commands/ArchivedCommitsCommand.java | 26 +- .../hoodie/cli/commands/CleansCommand.java | 38 +- .../hoodie/cli/commands/CommitsCommand.java | 148 ++- .../hoodie/cli/commands/DatasetsCommand.java | 7 +- .../commands/HDFSParquetImportCommand.java | 51 +- .../cli/commands/HoodieLogFileCommand.java | 80 +- .../cli/commands/HoodieSyncCommand.java | 76 +- .../hoodie/cli/commands/RepairsCommand.java | 40 +- .../cli/commands/SavepointsCommand.java | 48 +- .../uber/hoodie/cli/commands/SparkMain.java | 48 +- .../hoodie/cli/commands/StatsCommand.java | 62 +- .../hoodie/cli/commands/UtilsCommand.java | 4 +- .../com/uber/hoodie/cli/utils/CommitUtil.java | 11 +- .../com/uber/hoodie/cli/utils/HiveUtil.java | 39 +- .../hoodie/cli/utils/InputStreamConsumer.java | 2 +- .../com/uber/hoodie/cli/utils/SparkUtil.java | 12 +- .../com/uber/hoodie/HoodieReadClient.java | 46 +- .../com/uber/hoodie/HoodieWriteClient.java | 364 +++----- .../java/com/uber/hoodie/WriteStatus.java | 20 +- .../hoodie/config/DefaultHoodieConfig.java | 8 +- .../hoodie/config/HoodieCompactionConfig.java | 97 +- .../uber/hoodie/config/HoodieIndexConfig.java | 28 +- .../hoodie/config/HoodieMemoryConfig.java | 26 +- .../hoodie/config/HoodieMetricsConfig.java | 40 +- .../hoodie/config/HoodieStorageConfig.java | 5 +- .../uber/hoodie/config/HoodieWriteConfig.java | 54 +- .../uber/hoodie/func/BufferedIterator.java | 96 +- .../hoodie/func/BulkInsertMapFunction.java | 7 +- .../uber/hoodie/func/LazyInsertIterable.java | 83 +- .../hoodie/func/LazyIterableIterator.java | 4 +- .../com/uber/hoodie/index/HoodieIndex.java | 43 +- .../uber/hoodie/index/InMemoryHashIndex.java | 46 +- .../index/bloom/BloomIndexFileInfo.java | 8 +- .../hoodie/index/bloom/HoodieBloomIndex.java | 202 ++-- .../bloom/HoodieBloomIndexCheckFunction.java | 24 +- .../hoodie/index/bucketed/BucketedIndex.java | 4 +- .../uber/hoodie/index/hbase/HBaseIndex.java | 199 ++-- .../uber/hoodie/io/HoodieAppendHandle.java | 112 ++- .../com/uber/hoodie/io/HoodieCleanHelper.java | 62 +- .../hoodie/io/HoodieCommitArchiveLog.java | 49 +- .../uber/hoodie/io/HoodieCreateHandle.java | 31 +- .../com/uber/hoodie/io/HoodieIOHandle.java | 61 +- .../com/uber/hoodie/io/HoodieMergeHandle.java | 101 +- .../io/compact/CompactionOperation.java | 4 +- .../hoodie/io/compact/HoodieCompactor.java | 8 +- .../compact/HoodieRealtimeTableCompactor.java | 60 +- .../strategy/BoundedIOCompactionStrategy.java | 9 +- .../compact/strategy/CompactionStrategy.java | 8 +- .../strategy/DayBasedCompactionStrategy.java | 18 +- .../LogFileSizeBasedCompactionStrategy.java | 6 +- .../io/storage/HoodieParquetWriter.java | 56 +- .../storage/HoodieStorageWriterFactory.java | 18 +- .../io/storage/HoodieWrapperFileSystem.java | 177 ++-- .../uber/hoodie/metrics/HoodieMetrics.java | 25 +- .../java/com/uber/hoodie/metrics/Metrics.java | 2 +- .../metrics/MetricsGraphiteReporter.java | 19 +- .../metrics/MetricsReporterFactory.java | 3 +- .../hoodie/metrics/MetricsReporterType.java | 3 +- .../hoodie/table/HoodieCopyOnWriteTable.java | 870 +++++++++--------- .../hoodie/table/HoodieMergeOnReadTable.java | 215 ++--- .../com/uber/hoodie/table/HoodieTable.java | 47 +- .../UserDefinedBulkInsertPartitioner.java | 1 + .../uber/hoodie/table/WorkloadProfile.java | 11 +- .../com/uber/hoodie/table/WorkloadStat.java | 3 +- .../src/test/java/HoodieClientExample.java | 30 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 565 +++++------- .../java/com/uber/hoodie/TestMultiFS.java | 23 +- .../hoodie/common/HoodieClientTestUtils.java | 69 +- .../common/HoodieMergeOnReadTestUtils.java | 13 +- .../common/HoodieTestDataGenerator.java | 177 ++-- .../hoodie/common/TestRawTripPayload.java | 72 +- .../hoodie/config/HoodieWriteConfigTest.java | 3 +- .../hoodie/func/TestBufferedIterator.java | 324 ++++--- .../hoodie/func/TestUpdateMapFunction.java | 43 +- .../com/uber/hoodie/index/TestHbaseIndex.java | 96 +- .../uber/hoodie/index/TestHoodieIndex.java | 12 +- .../index/bloom/TestHoodieBloomIndex.java | 249 +++-- .../hoodie/io/TestHoodieCommitArchiveLog.java | 66 +- .../uber/hoodie/io/TestHoodieCompactor.java | 55 +- .../TestHoodieCompactionStrategy.java | 74 +- .../io/strategy/TestHoodieDataFile.java | 9 +- .../hoodie/io/strategy/TestHoodieLogFile.java | 8 +- .../hoodie/metrics/TestHoodieMetrics.java | 3 +- .../hoodie/table/TestCopyOnWriteTable.java | 204 ++-- .../hoodie/table/TestMergeOnReadTable.java | 197 ++-- .../hoodie/avro/HoodieAvroWriteSupport.java | 6 +- .../hoodie/avro/MercifulJsonConverter.java | 15 +- .../uber/hoodie/common/HoodieCleanStat.java | 5 +- .../common/model/HoodieCommitMetadata.java | 12 +- .../hoodie/common/model/HoodieFileGroup.java | 11 +- .../uber/hoodie/common/model/HoodieKey.java | 6 +- .../hoodie/common/model/HoodieLogFile.java | 10 +- .../common/model/HoodiePartitionMetadata.java | 5 +- .../hoodie/common/model/HoodieRecord.java | 8 +- .../common/model/HoodieRecordLocation.java | 4 +- .../common/model/HoodieRecordPayload.java | 4 +- .../hoodie/common/model/HoodieTableType.java | 10 +- .../hoodie/common/model/HoodieWriteStat.java | 3 +- .../storage/SizeAwareDataInputStream.java | 56 +- .../common/table/HoodieTableConfig.java | 2 +- .../common/table/HoodieTableMetaClient.java | 2 +- .../log/HoodieCompactedLogRecordScanner.java | 109 ++- .../common/table/log/HoodieLogFileReader.java | 106 +-- .../common/table/log/HoodieLogFormat.java | 68 +- .../table/log/HoodieLogFormatReader.java | 24 +- ...rsion.java => HoodieLogFormatVersion.java} | 45 +- .../table/log/HoodieLogFormatWriter.java | 56 +- .../table/log/block/HoodieAvroDataBlock.java | 14 +- .../log/block/HoodieAvroDataBlockVersion.java | 37 + .../table/log/block/HoodieCommandBlock.java | 7 +- .../log/block/HoodieCommandBlockVersion.java | 28 + .../table/log/block/HoodieCorruptBlock.java | 6 +- .../table/log/block/HoodieDeleteBlock.java | 14 +- .../log/block/HoodieDeleteBlockVersion.java | 28 + .../table/log/block/HoodieLogBlock.java | 57 +- .../log/block/HoodieLogBlockVersion.java | 33 + .../table/log/block/LogBlockVersion.java | 79 -- .../table/timeline/HoodieActiveTimeline.java | 32 +- .../timeline/HoodieArchivedTimeline.java | 8 +- .../table/timeline/HoodieDefaultTimeline.java | 12 +- .../common/table/timeline/HoodieInstant.java | 31 +- .../table/view/HoodieTableFileSystemView.java | 400 ++++---- .../com/uber/hoodie/common/util/FSUtils.java | 12 +- .../hoodie/common/util/HoodieAvroUtils.java | 4 +- .../uber/hoodie/common/util/ParquetUtils.java | 8 +- .../hoodie/common/util/ReflectionUtils.java | 3 +- .../hoodie/common/util/SpillableMapUtils.java | 58 +- .../common/util/collection/DiskBasedMap.java | 206 ++--- .../util/collection/ExternalSpillableMap.java | 62 +- .../util/collection/LazyFileIterable.java | 6 +- .../converter/HoodieRecordConverter.java | 5 - .../io/storage/SizeAwareDataOutputStream.java | 5 +- .../HoodieCorruptedDataException.java | 3 +- .../uber/hoodie/common/TestBloomFilter.java | 4 +- .../common/minicluster/HdfsTestService.java | 33 +- .../minicluster/ZookeeperTestService.java | 10 +- .../common/model/AvroBinaryTestPayload.java | 3 +- .../hoodie/common/model/HoodieTestUtils.java | 147 ++- .../common/model/TestHoodieWriteStat.java | 8 +- .../table/HoodieTableMetaClientTest.java | 37 +- .../common/table/log/HoodieLogFormatTest.java | 544 +++++------ .../string/HoodieActiveTimelineTest.java | 58 +- .../table/string/MockHoodieTimeline.java | 9 +- .../view/HoodieTableFileSystemViewTest.java | 253 ++--- .../hoodie/common/util/SchemaTestUtil.java | 50 +- .../common/util/SpillableMapTestUtils.java | 7 +- .../uber/hoodie/common/util/TestFSUtils.java | 9 +- .../hoodie/common/util/TestParquetUtils.java | 17 +- .../util/collection/TestDiskBasedMap.java | 21 +- .../collection/TestExternalSpillableMap.java | 45 +- .../uber/hoodie/hadoop/HoodieHiveUtil.java | 3 +- .../uber/hoodie/hadoop/HoodieInputFormat.java | 61 +- .../hadoop/HoodieROTablePathFilter.java | 32 +- .../realtime/HoodieRealtimeInputFormat.java | 38 +- .../realtime/HoodieRealtimeRecordReader.java | 70 +- .../uber/hoodie/hadoop/AnnotationTest.java | 1 - .../hoodie/hadoop/HoodieInputFormatTest.java | 63 +- .../hoodie/hadoop/InputFormatTestUtil.java | 31 +- .../hadoop/TestHoodieROTablePathFilter.java | 1 + .../HoodieRealtimeRecordReaderTest.java | 37 +- .../com/uber/hoodie/hive/HiveSyncConfig.java | 14 +- .../com/uber/hoodie/hive/HiveSyncTool.java | 35 +- .../uber/hoodie/hive/HoodieHiveClient.java | 147 ++- .../hoodie/hive/PartitionValueExtractor.java | 5 +- ...lashEncodedDayPartitionValueExtractor.java | 2 +- .../hoodie/hive/util/ColumnNameXLator.java | 4 +- .../com/uber/hoodie/hive/util/SchemaUtil.java | 88 +- .../uber/hoodie/hive/HiveSyncToolTest.java | 160 ++-- .../java/com/uber/hoodie/hive/TestUtil.java | 82 +- .../hoodie/hive/util/HiveTestService.java | 42 +- .../java/com/uber/hoodie/BaseAvroPayload.java | 1 - .../java/com/uber/hoodie/DataSourceUtils.java | 47 +- .../uber/hoodie/HoodieDataSourceHelpers.java | 5 +- .../java/com/uber/hoodie/KeyGenerator.java | 3 +- .../OverwriteWithLatestAvroPayload.java | 3 +- .../com/uber/hoodie/AvroConversionUtils.scala | 9 +- .../src/test/java/DataSourceTestUtils.java | 10 +- hoodie-spark/src/test/java/HoodieJavaApp.java | 45 +- .../hoodie/utilities/HDFSParquetImporter.java | 102 +- .../utilities/HiveIncrementalPuller.java | 64 +- .../utilities/HoodieSnapshotCopier.java | 67 +- .../uber/hoodie/utilities/UtilHelpers.java | 9 +- .../deltastreamer/HoodieDeltaStreamer.java | 145 +-- .../keygen/TimestampBasedKeyGenerator.java | 15 +- .../schema/FilebasedSchemaProvider.java | 16 +- .../hoodie/utilities/sources/DFSSource.java | 24 +- .../utilities/sources/HiveIncrPullSource.java | 12 +- .../hoodie/utilities/sources/KafkaSource.java | 39 +- .../uber/hoodie/utilities/sources/Source.java | 3 +- .../utilities/TestHDFSParquetImporter.java | 24 +- .../utilities/TestHoodieSnapshotCopier.java | 6 +- pom.xml | 35 + style/checkstyle-suppressions.xml | 12 + style/checkstyle.xml | 246 +++++ style/eclipse-java-google-style.xml | 337 +++++++ style/intellij-java-google-style.xml | 598 ++++++++++++ 200 files changed, 6209 insertions(+), 5975 deletions(-) rename hoodie-common/src/main/java/com/uber/hoodie/common/table/log/{LogFormatVersion.java => HoodieLogFormatVersion.java} (68%) create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlockVersion.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlockVersion.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlockVersion.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlockVersion.java delete mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java create mode 100644 style/checkstyle-suppressions.xml create mode 100644 style/checkstyle.xml create mode 100644 style/eclipse-java-google-style.xml create mode 100644 style/intellij-java-google-style.xml diff --git a/docs/dev_setup.md b/docs/dev_setup.md index 2fc34677f..1bdeec793 100644 --- a/docs/dev_setup.md +++ b/docs/dev_setup.md @@ -7,8 +7,7 @@ permalink: dev_setup.html ### Code Style - We have embraced the [Google Java code style](https://google.github.io/styleguide/javaguide.html). Please setup your IDE accordingly with style files from [here](https://github.com/google/styleguide/blob/gh-pages/intellij-java-google-style.xml) - Also recommend setting up the [Save Action Plugin](https://plugins.jetbrains.com/plugin/7642-save-actions) to auto format & organize imports on save. - - - + We have embraced the code style largely based on [google format](https://google.github.io/styleguide/javaguide.html). + Please setup your IDE with style files from [here](../style/) + We also recommend setting up the [Save Action Plugin](https://plugins.jetbrains.com/plugin/7642-save-actions) to auto format & organize imports on save. + The Maven Compilation life-cycle will fail if there are checkstyle violations. diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java index 2839cac99..937fd064b 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java @@ -35,10 +35,10 @@ public class HoodiePrompt extends DefaultPromptProvider { case DATASET: return "hoodie:" + tableName + "->"; case SYNC: - return "hoodie:" + tableName + " <==> " - + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->"; + return "hoodie:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->"; + default: + return "hoodie:" + tableName + "->"; } - return "hoodie:" + tableName + "->"; } return "hoodie->"; } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java index d6a16891a..6bbe2e218 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java @@ -24,22 +24,18 @@ import org.springframework.stereotype.Component; @Component @Order(Ordered.HIGHEST_PRECEDENCE) -public class HoodieSplashScreen - extends DefaultBannerProvider { +public class HoodieSplashScreen extends DefaultBannerProvider { - private static String screen = - "============================================" + OsUtils.LINE_SEPARATOR + - "* *" + OsUtils.LINE_SEPARATOR + - "* _ _ _ _ *" + OsUtils.LINE_SEPARATOR + - "* | | | | | (_) *" + OsUtils.LINE_SEPARATOR + - "* | |__| | ___ ___ __| |_ ___ *" + OsUtils.LINE_SEPARATOR + - "* | __ |/ _ \\ / _ \\ / _` | |/ _ \\ *" + - OsUtils.LINE_SEPARATOR + - "* | | | | (_) | (_) | (_| | | __/ *" + OsUtils.LINE_SEPARATOR + - "* |_| |_|\\___/ \\___/ \\__,_|_|\\___| *" + - OsUtils.LINE_SEPARATOR + - "* *" + OsUtils.LINE_SEPARATOR + - "============================================" + OsUtils.LINE_SEPARATOR; + private static String screen = "============================================" + OsUtils.LINE_SEPARATOR + + "* *" + OsUtils.LINE_SEPARATOR + + "* _ _ _ _ *" + OsUtils.LINE_SEPARATOR + + "* | | | | | (_) *" + OsUtils.LINE_SEPARATOR + + "* | |__| | ___ ___ __| |_ ___ *" + OsUtils.LINE_SEPARATOR + + "* | __ |/ _ \\ / _ \\ / _` | |/ _ \\ *" + OsUtils.LINE_SEPARATOR + + "* | | | | (_) | (_) | (_| | | __/ *" + OsUtils.LINE_SEPARATOR + + "* |_| |_|\\___/ \\___/ \\__,_|_|\\___| *" + OsUtils.LINE_SEPARATOR + + "* *" + OsUtils.LINE_SEPARATOR + + "============================================" + OsUtils.LINE_SEPARATOR; public String getBanner() { return screen; diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java index c0d7924ff..65f98f00f 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java @@ -22,8 +22,7 @@ import org.springframework.shell.Bootstrap; public class Main { /** - * Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging - * inside an IDE + * Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging inside an IDE */ public static void main(String[] args) throws IOException { Bootstrap.main(args); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java index 50fb6a565..99c79863b 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java @@ -47,13 +47,11 @@ public class ArchivedCommitsCommand implements CommandMarker { } @CliCommand(value = "show archived commits", help = "Read commits from archived files and show details") - public String showCommits( - @CliOption(key = { - "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") - final Integer limit) throws IOException { + public String showCommits(@CliOption(key = { + "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit) + throws IOException { - System.out - .println("===============> Showing only " + limit + " archived commits <==============="); + System.out.println("===============> Showing only " + limit + " archived commits <==============="); String basePath = HoodieCLI.tableMetadata.getBasePath(); FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf) .globStatus(new Path(basePath + "/.hoodie/.commits_.archive*")); @@ -61,8 +59,7 @@ public class ArchivedCommitsCommand implements CommandMarker { int commits = 0; for (FileStatus fs : fsStatuses) { //read the archived file - HoodieLogFormat.Reader reader = HoodieLogFormat - .newReader(FSUtils.getFs(basePath, HoodieCLI.conf), + HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf), new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema()); List readRecords = new ArrayList<>(); @@ -71,20 +68,19 @@ public class ArchivedCommitsCommand implements CommandMarker { HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); List records = blk.getRecords(); readRecords.addAll(records); - if(commits == limit) { + if (commits == limit) { break; } commits++; } - List readCommits = readRecords.stream().map(r -> (GenericRecord) r) - .map(r -> readCommit(r)).collect(Collectors.toList()); + List readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> readCommit(r)) + .collect(Collectors.toList()); allCommits.addAll(readCommits); - if(commits == limit) { + if (commits == limit) { break; } } - return HoodiePrintHelper.print( - new String[]{"CommitTime", "CommitType", "CommitDetails"}, + return HoodiePrintHelper.print(new String[] {"CommitTime", "CommitType", "CommitDetails"}, allCommits.toArray(new String[allCommits.size()][])); } @@ -122,6 +118,8 @@ public class ArchivedCommitsCommand implements CommandMarker { commitDetails.add(record.get("hoodieSavePointMetadata").toString()); break; } + default: + return commitDetails.toArray(new String[commitDetails.size()]); } } catch (Exception e) { e.printStackTrace(); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java index 38638866b..aa1792ecb 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.cli.commands; import com.uber.hoodie.avro.model.HoodieCleanMetadata; @@ -63,51 +64,46 @@ public class CleansCommand implements CommandMarker { Collections.reverse(cleans); for (int i = 0; i < cleans.size(); i++) { HoodieInstant clean = cleans.get(i); - HoodieCleanMetadata cleanMetadata = - AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get()); - rows[i] = new String[]{clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(), - String.valueOf(cleanMetadata.getTotalFilesDeleted()), - String.valueOf(cleanMetadata.getTimeTakenInMillis())}; + HoodieCleanMetadata cleanMetadata = AvroUtils + .deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get()); + rows[i] = new String[] {clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(), + String.valueOf(cleanMetadata.getTotalFilesDeleted()), String.valueOf(cleanMetadata.getTimeTakenInMillis())}; } - return HoodiePrintHelper.print( - new String[]{"CleanTime", "EarliestCommandRetained", "Total Files Deleted", - "Total Time Taken"}, rows); + return HoodiePrintHelper + .print(new String[] {"CleanTime", "EarliestCommandRetained", "Total Files Deleted", "Total Time Taken"}, + rows); } @CliCommand(value = "cleans refresh", help = "Refresh the commits") public String refreshCleans() throws IOException { - HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } @CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean") - public String showCleanPartitions( - @CliOption(key = {"clean"}, help = "clean to show") - final String commitTime) throws Exception { + public String showCleanPartitions(@CliOption(key = {"clean"}, help = "clean to show") final String commitTime) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants(); - HoodieInstant cleanInstant = - new HoodieInstant(false, HoodieTimeline.CLEAN_ACTION, commitTime); + HoodieInstant cleanInstant = new HoodieInstant(false, HoodieTimeline.CLEAN_ACTION, commitTime); if (!timeline.containsInstant(cleanInstant)) { return "Clean " + commitTime + " not found in metadata " + timeline; } - HoodieCleanMetadata cleanMetadata = - AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(cleanInstant).get()); + HoodieCleanMetadata cleanMetadata = AvroUtils.deserializeHoodieCleanMetadata( + timeline.getInstantDetails(cleanInstant).get()); List rows = new ArrayList<>(); - for (Map.Entry entry : cleanMetadata - .getPartitionMetadata().entrySet()) { + for (Map.Entry entry : cleanMetadata.getPartitionMetadata().entrySet()) { String path = entry.getKey(); HoodieCleanPartitionMetadata stats = entry.getValue(); String policy = stats.getPolicy(); String totalSuccessDeletedFiles = String.valueOf(stats.getSuccessDeleteFiles().size()); String totalFailedDeletedFiles = String.valueOf(stats.getFailedDeleteFiles().size()); - rows.add(new String[]{path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles}); + rows.add(new String[] {path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles}); } return HoodiePrintHelper.print( - new String[]{"Partition Path", "Cleaning policy", "Total Files Successfully Deleted", + new String[] {"Partition Path", "Cleaning policy", "Total Files Successfully Deleted", "Total Failed Deletions"}, rows.toArray(new String[rows.size()][])); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java index d7b6594d1..0f5f87b09 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java @@ -64,21 +64,18 @@ public class CommitsCommand implements CommandMarker { } @CliCommand(value = "commits show", help = "Show the commits") - public String showCommits( - @CliOption(key = { - "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") - final Integer limit) throws IOException { + public String showCommits(@CliOption(key = { + "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit) + throws IOException { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); List commits = timeline.getInstants().collect(Collectors.toList()); String[][] rows = new String[commits.size()][]; Collections.reverse(commits); for (int i = 0; i < commits.size(); i++) { HoodieInstant commit = commits.get(i); - HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get()); - rows[i] = new String[]{commit.getTimestamp(), + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get()); + rows[i] = new String[] {commit.getTimestamp(), NumericUtils.humanReadableByteCount(commitMetadata.fetchTotalBytesWritten()), String.valueOf(commitMetadata.fetchTotalFilesInsert()), String.valueOf(commitMetadata.fetchTotalFilesUpdated()), @@ -88,39 +85,32 @@ public class CommitsCommand implements CommandMarker { String.valueOf(commitMetadata.fetchTotalWriteErrors())}; } return HoodiePrintHelper.print( - new String[]{"CommitTime", "Total Written (B)", "Total Files Added", - "Total Files Updated", "Total Partitions Written", "Total Records Written", - "Total Update Records Written", "Total Errors"}, rows); + new String[] {"CommitTime", "Total Written (B)", "Total Files Added", "Total Files Updated", + "Total Partitions Written", "Total Records Written", "Total Update Records Written", "Total Errors"}, rows); } @CliCommand(value = "commits refresh", help = "Refresh the commits") public String refreshCommits() throws IOException { - HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } @CliCommand(value = "commit rollback", help = "Rollback a commit") - public String rollbackCommit( - @CliOption(key = {"commit"}, help = "Commit to rollback") - final String commitTime, - @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") - final String sparkPropertiesPath) throws Exception { + public String rollbackCommit(@CliOption(key = {"commit"}, help = "Commit to rollback") final String commitTime, + @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") final String sparkPropertiesPath) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsTimeline() - .filterCompletedInstants(); - HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, - commitTime); + HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; } SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); - sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(), - commitTime, - HoodieCLI.tableMetadata.getBasePath()); + sparkLauncher + .addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(), commitTime, HoodieCLI.tableMetadata.getBasePath()); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); int exitCode = process.waitFor(); @@ -133,23 +123,18 @@ public class CommitsCommand implements CommandMarker { } @CliCommand(value = "commit showpartitions", help = "Show partition level details of a commit") - public String showCommitPartitions( - @CliOption(key = {"commit"}, help = "Commit to show") - final String commitTime) throws Exception { + public String showCommitPartitions(@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsTimeline() - .filterCompletedInstants(); - HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, - commitTime); + HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; } - HoodieCommitMetadata meta = - HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); + HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); List rows = new ArrayList(); - for (Map.Entry> entry : meta.getPartitionToWriteStats() - .entrySet()) { + for (Map.Entry> entry : meta.getPartitionToWriteStats().entrySet()) { String path = entry.getKey(); List stats = entry.getValue(); long totalFilesAdded = 0; @@ -169,50 +154,40 @@ public class CommitsCommand implements CommandMarker { totalBytesWritten += stat.getTotalWriteBytes(); totalWriteErrors += stat.getTotalWriteErrors(); } - rows.add(new String[]{path, String.valueOf(totalFilesAdded), - String.valueOf(totalFilesUpdated), String.valueOf(totalRecordsInserted), - String.valueOf(totalRecordsUpdated), - NumericUtils.humanReadableByteCount(totalBytesWritten), - String.valueOf(totalWriteErrors)}); + rows.add(new String[] {path, String.valueOf(totalFilesAdded), String.valueOf(totalFilesUpdated), + String.valueOf(totalRecordsInserted), String.valueOf(totalRecordsUpdated), + NumericUtils.humanReadableByteCount(totalBytesWritten), String.valueOf(totalWriteErrors)}); } return HoodiePrintHelper.print( - new String[]{"Partition Path", "Total Files Added", "Total Files Updated", - "Total Records Inserted", "Total Records Updated", "Total Bytes Written", - "Total Errors"}, rows.toArray(new String[rows.size()][])); + new String[] {"Partition Path", "Total Files Added", "Total Files Updated", "Total Records Inserted", + "Total Records Updated", "Total Bytes Written", "Total Errors"}, rows.toArray(new String[rows.size()][])); } @CliCommand(value = "commit showfiles", help = "Show file level details of a commit") - public String showCommitFiles( - @CliOption(key = {"commit"}, help = "Commit to show") - final String commitTime) throws Exception { + public String showCommitFiles(@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); - HoodieTimeline timeline = activeTimeline.getCommitsTimeline() - .filterCompletedInstants(); - HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, - commitTime); + HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; } - HoodieCommitMetadata meta = - HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); + HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get()); List rows = new ArrayList(); - for (Map.Entry> entry : meta.getPartitionToWriteStats() - .entrySet()) { + for (Map.Entry> entry : meta.getPartitionToWriteStats().entrySet()) { String path = entry.getKey(); List stats = entry.getValue(); for (HoodieWriteStat stat : stats) { - rows.add(new String[]{path, stat.getFileId(), stat.getPrevCommit(), - String.valueOf(stat.getNumUpdateWrites()), String.valueOf(stat.getNumWrites()), - String.valueOf(stat.getTotalWriteBytes()), + rows.add(new String[] {path, stat.getFileId(), stat.getPrevCommit(), String.valueOf(stat.getNumUpdateWrites()), + String.valueOf(stat.getNumWrites()), String.valueOf(stat.getTotalWriteBytes()), String.valueOf(stat.getTotalWriteErrors())}); } } return HoodiePrintHelper.print( - new String[]{"Partition Path", "File ID", "Previous Commit", "Total Records Updated", - "Total Records Written", "Total Bytes Written", "Total Errors"}, - rows.toArray(new String[rows.size()][])); + new String[] {"Partition Path", "File ID", "Previous Commit", "Total Records Updated", "Total Records Written", + "Total Bytes Written", "Total Errors"}, rows.toArray(new String[rows.size()][])); } @CliAvailabilityIndicator({"commits compare"}) @@ -221,38 +196,30 @@ public class CommitsCommand implements CommandMarker { } @CliCommand(value = "commits compare", help = "Compare commits with another Hoodie dataset") - public String compareCommits( - @CliOption(key = {"path"}, help = "Path of the dataset to compare to") - final String path) throws Exception { + public String compareCommits(@CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) + throws Exception { HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.conf, path); - HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieTableMetaClient source = HoodieCLI.tableMetadata; - HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); String targetLatestCommit = - targetTimeline.getInstants().iterator().hasNext() ? "0" - : targetTimeline.lastInstant().get().getTimestamp(); + targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp(); String sourceLatestCommit = - sourceTimeline.getInstants().iterator().hasNext() ? "0" - : sourceTimeline.lastInstant().get().getTimestamp(); + sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp(); - if (sourceLatestCommit != null && - HoodieTimeline - .compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) { + if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit, + HoodieTimeline.GREATER)) { // source is behind the target - List commitsToCatchup = - targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) - .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - return "Source " + source.getTableConfig().getTableName() + " is behind by " - + commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup; + List commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) + .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + return "Source " + source.getTableConfig().getTableName() + " is behind by " + commitsToCatchup.size() + + " commits. Commits to catch up - " + commitsToCatchup; } else { - List commitsToCatchup = - sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE) - .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - return "Source " + source.getTableConfig().getTableName() + " is ahead by " - + commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup; + List commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE) + .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + return "Source " + source.getTableConfig().getTableName() + " is ahead by " + commitsToCatchup.size() + + " commits. Commits to catch up - " + commitsToCatchup; } } @@ -262,13 +229,12 @@ public class CommitsCommand implements CommandMarker { } @CliCommand(value = "commits sync", help = "Compare commits with another Hoodie dataset") - public String syncCommits( - @CliOption(key = {"path"}, help = "Path of the dataset to compare to") - final String path) throws Exception { + public String syncCommits(@CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) + throws Exception { HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.conf, path); HoodieCLI.state = HoodieCLI.CLIState.SYNC; - return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName() - + " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName(); + return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " and " + + HoodieCLI.syncTableMetadata.getTableConfig().getTableName(); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java index de59d0669..1bcdfa5bc 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java @@ -29,13 +29,12 @@ public class DatasetsCommand implements CommandMarker { @CliCommand(value = "connect", help = "Connect to a hoodie dataset") public String connect( - @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") - final String path) throws IOException { + @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path) + throws IOException { boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.conf, path)); HoodieCLI.state = HoodieCLI.CLIState.DATASET; - return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() - + " loaded"; + return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded"; } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java index c9d2f98b5..c3298c9ae 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HDFSParquetImportCommand.java @@ -37,44 +37,33 @@ public class HDFSParquetImportCommand implements CommandMarker { @CliCommand(value = "hdfsparquetimport", help = "Imports hdfs dataset to a hoodie dataset") public String convert( - @CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset") - final String srcPath, - @CliOption(key = "srcType", mandatory = true, help = "Source type for the input dataset") - final String srcType, - @CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset") - final String targetPath, - @CliOption(key = "tableName", mandatory = true, help = "Table name") - final String tableName, - @CliOption(key = "tableType", mandatory = true, help = "Table type") - final String tableType, - @CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") - final String rowKeyField, - @CliOption(key = "partitionPathField", mandatory = true, help = "Partition path field name") - final String partitionPathField, - @CliOption(key = {"parallelism"}, mandatory = true, help = "Parallelism for hoodie insert") - final String parallelism, - @CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") - final String schemaFilePath, - @CliOption(key = "format", mandatory = true, help = "Format for the input data") - final String format, - @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") - final String sparkMemory, - @CliOption(key = "retry", mandatory = true, help = "Number of retries") - final String retry) - throws Exception { + @CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset") final String srcPath, + @CliOption(key = "srcType", mandatory = true, help = "Source type for the input dataset") final String srcType, + @CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset") final String + targetPath, + @CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName, + @CliOption(key = "tableType", mandatory = true, help = "Table type") final String tableType, + @CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField, + @CliOption(key = "partitionPathField", mandatory = true, help = "Partition path field name") final String + partitionPathField, + @CliOption(key = { + "parallelism"}, mandatory = true, help = "Parallelism for hoodie insert") final String parallelism, + @CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") final String + schemaFilePath, + @CliOption(key = "format", mandatory = true, help = "Format for the input data") final String format, + @CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory, + @CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry) throws Exception { validate(format, srcType); boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); - String sparkPropertiesPath = Utils - .getDefaultPropertiesFile( - scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); + String sparkPropertiesPath = Utils.getDefaultPropertiesFile( + scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties())); SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); - sparkLauncher.addAppArgs(SparkCommand.IMPORT.toString(), srcPath, targetPath, tableName, - tableType, rowKeyField, partitionPathField, parallelism, schemaFilePath, sparkMemory, - retry); + sparkLauncher.addAppArgs(SparkCommand.IMPORT.toString(), srcPath, targetPath, tableName, tableType, rowKeyField, + partitionPathField, parallelism, schemaFilePath, sparkMemory, retry); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); int exitCode = process.waitFor(); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java index 2de48500b..9d9790bba 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -64,25 +64,25 @@ public class HoodieLogFileCommand implements CommandMarker { @CliCommand(value = "show logfile metadata", help = "Read commit metadata from log files") public String showLogFileCommits( - @CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified path for the log file") - final String logFilePathPattern) throws IOException { + @CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified path for the log file") final + String logFilePathPattern) + throws IOException { FileSystem fs = HoodieCLI.tableMetadata.getFs(); List logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern))) .map(status -> status.getPath().toString()).collect(Collectors.toList()); - Map, Map>, Integer>>> commitCountAndMetadata = Maps - .newHashMap(); + Map, Map>, Integer>>> + commitCountAndMetadata = Maps.newHashMap(); int totalEntries = 0; int numCorruptBlocks = 0; for (String logFilePath : logFilePaths) { - FileStatus[] fsStatus = fs.listStatus( - new Path(logFilePath)); - Schema writerSchema = new AvroSchemaConverter() - .convert(SchemaUtil - .readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFilePath))); - HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, - new HoodieLogFile(fsStatus[0].getPath()), writerSchema); + FileStatus[] fsStatus = fs.listStatus(new Path(logFilePath)); + Schema writerSchema = new AvroSchemaConverter().convert( + SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFilePath))); + HoodieLogFormat.Reader reader = HoodieLogFormat + .newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema); // read the avro blocks while (reader.hasNext()) { @@ -104,15 +104,14 @@ public class HoodieLogFileCommand implements CommandMarker { } } if (commitCountAndMetadata.containsKey(instantTime)) { - commitCountAndMetadata.get(instantTime) - .add(new Tuple3<>(n.getBlockType(), - new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount)); + commitCountAndMetadata.get(instantTime).add( + new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount)); totalEntries++; } else { - List, Map>, Integer>> list - = new ArrayList<>(); - list.add(new Tuple3<>(n.getBlockType(), - new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount)); + List, Map>, + Integer>> list = new ArrayList<>(); + list.add( + new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount)); commitCountAndMetadata.put(instantTime, list); totalEntries++; } @@ -121,11 +120,12 @@ public class HoodieLogFileCommand implements CommandMarker { String[][] rows = new String[totalEntries + 1][]; int i = 0; ObjectMapper objectMapper = new ObjectMapper(); - for (Map.Entry, Map>, Integer>>> entry : commitCountAndMetadata - .entrySet()) { + for (Map.Entry, Map>, Integer>>> entry + : commitCountAndMetadata.entrySet()) { String instantTime = entry.getKey().toString(); - for (Tuple3, Map>, Integer> tuple3 : entry - .getValue()) { + for (Tuple3, + Map>, Integer> tuple3 : entry.getValue()) { String[] output = new String[5]; output[0] = instantTime; output[1] = String.valueOf(tuple3._3()); @@ -136,24 +136,21 @@ public class HoodieLogFileCommand implements CommandMarker { i++; } } - return HoodiePrintHelper.print( - new String[]{"InstantTime", "RecordCount", "BlockType", "HeaderMetadata", "FooterMetadata"}, - rows); + return HoodiePrintHelper + .print(new String[] {"InstantTime", "RecordCount", "BlockType", "HeaderMetadata", "FooterMetadata"}, + rows); } @CliCommand(value = "show logfile records", help = "Read records from log files") - public String showLogFileRecords( - @CliOption(key = { - "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") - final Integer limit, + public String showLogFileRecords(@CliOption(key = { + "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit, @CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified paths for the log files") final String logFilePathPattern, @CliOption(key = "mergeRecords", mandatory = false, help = "If the records in the log files should be merged", - unspecifiedDefaultValue = "false") - final Boolean shouldMerge) throws IOException { + unspecifiedDefaultValue = "false") final Boolean shouldMerge) + throws IOException { - System.out - .println("===============> Showing only " + limit + " records <==============="); + System.out.println("===============> Showing only " + limit + " records <==============="); FileSystem fs = HoodieCLI.tableMetadata.getFs(); List logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern))) @@ -162,9 +159,8 @@ public class HoodieLogFileCommand implements CommandMarker { // TODO : readerSchema can change across blocks/log files, fix this inside Scanner AvroSchemaConverter converter = new AvroSchemaConverter(); // get schema from last log file - Schema readerSchema = converter - .convert(SchemaUtil - .readSchemaFromLogFile(fs, new Path(logFilePaths.get(logFilePaths.size() - 1)))); + Schema readerSchema = converter.convert( + SchemaUtil.readSchemaFromLogFile(fs, new Path(logFilePaths.get(logFilePaths.size() - 1)))); List allRecords = new ArrayList<>(); @@ -186,11 +182,10 @@ public class HoodieLogFileCommand implements CommandMarker { } } else { for (String logFile : logFilePaths) { - Schema writerSchema = new AvroSchemaConverter() - .convert(SchemaUtil - .readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFile))); - HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, - new HoodieLogFile(new Path(logFile)), writerSchema); + Schema writerSchema = new AvroSchemaConverter().convert( + SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFile))); + HoodieLogFormat.Reader reader = HoodieLogFormat + .newReader(fs, new HoodieLogFile(new Path(logFile)), writerSchema); // read the avro blocks while (reader.hasNext()) { HoodieLogBlock n = reader.next(); @@ -216,7 +211,6 @@ public class HoodieLogFileCommand implements CommandMarker { rows[i] = data; i++; } - return HoodiePrintHelper.print( - new String[]{"Records"}, rows); + return HoodiePrintHelper.print(new String[] {"Records"}, rows); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java index dd9560a4a..0640c9416 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java @@ -40,26 +40,22 @@ public class HoodieSyncCommand implements CommandMarker { @CliCommand(value = "sync validate", help = "Validate the sync by counting the number of records") public String validateSync( - @CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode") - final String mode, + @CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode") final String mode, + @CliOption(key = {"sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database") final String srcDb, @CliOption(key = { - "sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database") - final String srcDb, - @CliOption(key = { - "targetDb"}, unspecifiedDefaultValue = "dwh_hoodie", help = "target database") - final String tgtDb, + "targetDb"}, unspecifiedDefaultValue = "dwh_hoodie", help = "target database") final String tgtDb, @CliOption(key = { "partitionCount"}, unspecifiedDefaultValue = "5", help = "total number of recent partitions to validate") final int partitionCount, @CliOption(key = { - "hiveServerUrl"}, mandatory = true, help = "hiveServerURL to connect to") - final String hiveServerUrl, + "hiveServerUrl"}, mandatory = true, help = "hiveServerURL to connect to") final String hiveServerUrl, @CliOption(key = { - "hiveUser"}, mandatory = false, unspecifiedDefaultValue = "", help = "hive username to connect to") - final String hiveUser, + "hiveUser"}, mandatory = false, unspecifiedDefaultValue = "", help = "hive username to connect to") final + String hiveUser, @CliOption(key = { - "hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to") - final String hivePass) throws Exception { + "hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to") final + String hivePass) + throws Exception { HoodieTableMetaClient target = HoodieCLI.syncTableMetadata; HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline(); HoodieTableMetaClient source = HoodieCLI.tableMetadata; @@ -70,52 +66,42 @@ public class HoodieSyncCommand implements CommandMarker { sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, hiveUser, hivePass); targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, hiveUser, hivePass); } else if ("latestPartitions".equals(mode)) { - sourceCount = HiveUtil - .countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass); - targetCount = HiveUtil - .countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass); + sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass); + targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass); } String targetLatestCommit = - targetTimeline.getInstants().iterator().hasNext() ? "0" - : targetTimeline.lastInstant().get().getTimestamp(); + targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp(); String sourceLatestCommit = - sourceTimeline.getInstants().iterator().hasNext() ? "0" - : sourceTimeline.lastInstant().get().getTimestamp(); + sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp(); - if (sourceLatestCommit != null && HoodieTimeline - .compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) { + if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit, + HoodieTimeline.GREATER)) { // source is behind the target - List commitsToCatchup = - targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE).getInstants() - .collect(Collectors.toList()); + List commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) + .getInstants().collect(Collectors.toList()); if (commitsToCatchup.isEmpty()) { - return "Count difference now is (count(" + target.getTableConfig().getTableName() - + ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount - - sourceCount); + return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count(" + + source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount); } else { long newInserts = CommitUtil.countNewRecords(target, - commitsToCatchup.stream().map(HoodieInstant::getTimestamp) - .collect(Collectors.toList())); - return "Count difference now is (count(" + target.getTableConfig().getTableName() - + ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount - - sourceCount) + ". Catch up count is " + newInserts; + commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList())); + return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count(" + + source.getTableConfig().getTableName() + + ") == " + (targetCount - sourceCount) + ". Catch up count is " + newInserts; } } else { - List commitsToCatchup = - sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE).getInstants() - .collect(Collectors.toList()); + List commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE) + .getInstants().collect(Collectors.toList()); if (commitsToCatchup.isEmpty()) { - return "Count difference now is (count(" + source.getTableConfig().getTableName() - + ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount - - targetCount); + return "Count difference now is (count(" + source.getTableConfig().getTableName() + ") - count(" + + target.getTableConfig().getTableName() + ") == " + (sourceCount - targetCount); } else { long newInserts = CommitUtil.countNewRecords(source, - commitsToCatchup.stream().map(HoodieInstant::getTimestamp) - .collect(Collectors.toList())); - return "Count difference now is (count(" + source.getTableConfig().getTableName() - + ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount - - targetCount) + ". Catch up count is " + newInserts; + commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList())); + return "Count difference now is (count(" + source.getTableConfig().getTableName() + ") - count(" + + target.getTableConfig().getTableName() + + ") == " + (sourceCount - targetCount) + ". Catch up count is " + newInserts; } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RepairsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RepairsCommand.java index 1db6075e1..bf6c82888 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RepairsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RepairsCommand.java @@ -45,20 +45,20 @@ public class RepairsCommand implements CommandMarker { return HoodieCLI.tableMetadata != null; } - @CliCommand(value = "repair deduplicate", help = "De-duplicate a partition path contains duplicates & produce repaired files to replace with") - public String deduplicate( + @CliCommand(value = "repair deduplicate", help = "De-duplicate a partition path contains duplicates & produce " + + "repaired files to replace with") + public String deduplicate(@CliOption(key = { + "duplicatedPartitionPath"}, help = "Partition Path containing the duplicates", mandatory = true) final String + duplicatedPartitionPath, @CliOption(key = { - "duplicatedPartitionPath"}, help = "Partition Path containing the duplicates", mandatory = true) - final String duplicatedPartitionPath, + "repairedOutputPath"}, help = "Location to place the repaired files", mandatory = true) final String + repairedOutputPath, @CliOption(key = { - "repairedOutputPath"}, help = "Location to place the repaired files", mandatory = true) - final String repairedOutputPath, - @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path", mandatory = true) - final String sparkPropertiesPath) throws Exception { + "sparkProperties"}, help = "Spark Properites File Path", mandatory = true) final String sparkPropertiesPath) + throws Exception { SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); - sparkLauncher - .addAppArgs(SparkMain.SparkCommand.DEDUPLICATE.toString(), duplicatedPartitionPath, - repairedOutputPath, HoodieCLI.tableMetadata.getBasePath()); + sparkLauncher.addAppArgs(SparkMain.SparkCommand.DEDUPLICATE.toString(), duplicatedPartitionPath, repairedOutputPath, + HoodieCLI.tableMetadata.getBasePath()); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); int exitCode = process.waitFor(); @@ -71,14 +71,12 @@ public class RepairsCommand implements CommandMarker { @CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a dataset, if not present") - public String addPartitionMeta( - @CliOption(key = {"dryrun"}, - help = "Should we actually add or just print what would be done", - unspecifiedDefaultValue = "true") + public String addPartitionMeta(@CliOption(key = { + "dryrun"}, help = "Should we actually add or just print what would be done", unspecifiedDefaultValue = "true") final boolean dryRun) throws IOException { - String latestCommit = HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline() - .lastInstant().get().getTimestamp(); + String latestCommit = HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get() + .getTimestamp(); List partitionPaths = FSUtils.getAllFoldersThreeLevelsDown(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); Path basePath = new Path(HoodieCLI.tableMetadata.getBasePath()); @@ -94,10 +92,7 @@ public class RepairsCommand implements CommandMarker { if (!HoodiePartitionMetadata.hasPartitionMetadata(HoodieCLI.fs, partitionPath)) { row[1] = "No"; if (!dryRun) { - HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata( - HoodieCLI.fs, - latestCommit, - basePath, + HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath, partitionPath); partitionMetadata.trySave(0); } @@ -105,7 +100,6 @@ public class RepairsCommand implements CommandMarker { rows[ind++] = row; } - return HoodiePrintHelper.print( - new String[]{"Partition Path", "Metadata Present?", "Action"}, rows); + return HoodiePrintHelper.print(new String[] {"Partition Path", "Metadata Present?", "Action"}, rows); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java index db267d6a6..0bbfa3d1a 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.cli.commands; import com.uber.hoodie.HoodieWriteClient; @@ -60,8 +61,8 @@ public class SavepointsCommand implements CommandMarker { @CliAvailabilityIndicator({"savepoint rollback"}) public boolean isRollbackToSavepointAvailable() { - return HoodieCLI.tableMetadata != null && !HoodieCLI.tableMetadata.getActiveTimeline() - .getSavePointTimeline().filterCompletedInstants().empty(); + return HoodieCLI.tableMetadata != null && !HoodieCLI.tableMetadata.getActiveTimeline().getSavePointTimeline() + .filterCompletedInstants().empty(); } @CliCommand(value = "savepoints show", help = "Show the savepoints") @@ -73,23 +74,19 @@ public class SavepointsCommand implements CommandMarker { Collections.reverse(commits); for (int i = 0; i < commits.size(); i++) { HoodieInstant commit = commits.get(i); - rows[i] = new String[]{commit.getTimestamp()}; + rows[i] = new String[] {commit.getTimestamp()}; } - return HoodiePrintHelper.print(new String[]{"SavepointTime"}, rows); + return HoodiePrintHelper.print(new String[] {"SavepointTime"}, rows); } @CliCommand(value = "savepoint create", help = "Savepoint a commit") - public String savepoint( - @CliOption(key = {"commit"}, help = "Commit to savepoint") - final String commitTime, - @CliOption(key = {"user"}, help = "User who is creating the savepoint") - final String user, - @CliOption(key = {"comments"}, help = "Comments for creating the savepoint") - final String comments) throws Exception { + public String savepoint(@CliOption(key = {"commit"}, help = "Commit to savepoint") final String commitTime, + @CliOption(key = {"user"}, help = "User who is creating the savepoint") final String user, + @CliOption(key = {"comments"}, help = "Comments for creating the savepoint") final String comments) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants(); - HoodieInstant - commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; @@ -106,22 +103,19 @@ public class SavepointsCommand implements CommandMarker { @CliCommand(value = "savepoint rollback", help = "Savepoint a commit") public String rollbackToSavepoint( - @CliOption(key = {"savepoint"}, help = "Savepoint to rollback") - final String commitTime, - @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") - final String sparkPropertiesPath) throws Exception { + @CliOption(key = {"savepoint"}, help = "Savepoint to rollback") final String commitTime, + @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") final String sparkPropertiesPath) + throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants(); - HoodieInstant - commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!timeline.containsInstant(commitInstant)) { return "Commit " + commitTime + " not found in Commits " + timeline; } SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); - sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK_TO_SAVEPOINT.toString(), - commitTime, + sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK_TO_SAVEPOINT.toString(), commitTime, HoodieCLI.tableMetadata.getBasePath()); Process process = sparkLauncher.launch(); InputStreamConsumer.captureOutput(process); @@ -137,18 +131,14 @@ public class SavepointsCommand implements CommandMarker { @CliCommand(value = "savepoints refresh", help = "Refresh the savepoints") public String refreshMetaClient() throws IOException { - HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } - private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) - throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); + private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); return new HoodieWriteClient(jsc, config, false); } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java index 4f8f2f856..32e2a0091 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java @@ -30,18 +30,14 @@ import org.apache.spark.sql.SQLContext; public class SparkMain { - protected final static Logger LOG = Logger.getLogger(SparkMain.class); + protected static final Logger LOG = Logger.getLogger(SparkMain.class); /** * Commands */ enum SparkCommand { - ROLLBACK, - DEDUPLICATE, - ROLLBACK_TO_SAVEPOINT, - SAVEPOINT, - IMPORT + ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT } public static void main(String[] args) throws Exception { @@ -67,18 +63,19 @@ public class SparkMain { break; case IMPORT: assert (args.length == 11); - returnCode = dataImport(jsc, args[1], args[2], args[3], args[4], args[5], args[6], - Integer.parseInt(args[7]), args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9], - Integer.parseInt(args[10])); + returnCode = dataImport(jsc, args[1], args[2], args[3], args[4], args[5], args[6], Integer.parseInt(args[7]), + args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9], Integer.parseInt(args[10])); + break; + default: break; } System.exit(returnCode); } - private static int dataImport(JavaSparkContext jsc, String srcPath, String targetPath, - String tableName, String tableType, String rowKey, String partitionKey, int parallelism, - String schemaFile, String sparkMaster, String sparkMemory, int retry) throws Exception { + private static int dataImport(JavaSparkContext jsc, String srcPath, String targetPath, String tableName, + String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile, String sparkMaster, + String sparkMemory, int retry) throws Exception { HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config(); cfg.srcPath = srcPath; cfg.targetPath = targetPath; @@ -92,19 +89,15 @@ public class SparkMain { return new HDFSParquetImporter(cfg).dataImport(jsc, retry); } - private static int deduplicatePartitionPath(JavaSparkContext jsc, - String duplicatedPartitionPath, - String repairedOutputPath, - String basePath) - throws Exception { - DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, - new SQLContext(jsc), FSUtils.getFs(basePath, jsc.hadoopConfiguration())); + private static int deduplicatePartitionPath(JavaSparkContext jsc, String duplicatedPartitionPath, + String repairedOutputPath, String basePath) throws Exception { + DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc), + FSUtils.getFs(basePath, jsc.hadoopConfiguration())); job.fixDuplicates(true); return 0; } - private static int rollback(JavaSparkContext jsc, String commitTime, String basePath) - throws Exception { + private static int rollback(JavaSparkContext jsc, String commitTime, String basePath) throws Exception { HoodieWriteClient client = createHoodieClient(jsc, basePath); if (client.rollback(commitTime)) { LOG.info(String.format("The commit \"%s\" rolled back.", commitTime)); @@ -115,9 +108,7 @@ public class SparkMain { } } - private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, - String basePath) - throws Exception { + private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, String basePath) throws Exception { HoodieWriteClient client = createHoodieClient(jsc, basePath); if (client.rollbackToSavepoint(savepointTime)) { LOG.info(String.format("The commit \"%s\" rolled back.", savepointTime)); @@ -128,12 +119,9 @@ public class SparkMain { } } - private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) - throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); + private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); return new HoodieWriteClient(jsc, config); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java index cb61eef0b..2cdf826e8 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java @@ -16,7 +16,6 @@ package com.uber.hoodie.cli.commands; - import com.codahale.metrics.Histogram; import com.codahale.metrics.Snapshot; import com.codahale.metrics.UniformReservoir; @@ -44,12 +43,15 @@ import org.springframework.stereotype.Component; @Component public class StatsCommand implements CommandMarker { + private static final int MAX_FILES = 1000000; + @CliAvailabilityIndicator({"stats wa"}) public boolean isWriteAmpAvailable() { return HoodieCLI.tableMetadata != null; } - @CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many records were actually written") + @CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many " + + "records were actually written") public String writeAmplificationStats() throws IOException { long totalRecordsUpserted = 0; long totalRecordsWritten = 0; @@ -60,18 +62,13 @@ public class StatsCommand implements CommandMarker { String[][] rows = new String[new Long(timeline.countInstants()).intValue() + 1][]; int i = 0; DecimalFormat df = new DecimalFormat("#.00"); - for (HoodieInstant commitTime : timeline.getInstants().collect( - Collectors.toList())) { + for (HoodieInstant commitTime : timeline.getInstants().collect(Collectors.toList())) { String waf = "0"; - HoodieCommitMetadata commit = HoodieCommitMetadata - .fromBytes(activeTimeline.getInstantDetails(commitTime).get()); + HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitTime).get()); if (commit.fetchTotalUpdateRecordsWritten() > 0) { - waf = df.format( - (float) commit.fetchTotalRecordsWritten() / commit - .fetchTotalUpdateRecordsWritten()); + waf = df.format((float) commit.fetchTotalRecordsWritten() / commit.fetchTotalUpdateRecordsWritten()); } - rows[i++] = new String[]{commitTime.getTimestamp(), - String.valueOf(commit.fetchTotalUpdateRecordsWritten()), + rows[i++] = new String[] {commitTime.getTimestamp(), String.valueOf(commit.fetchTotalUpdateRecordsWritten()), String.valueOf(commit.fetchTotalRecordsWritten()), waf}; totalRecordsUpserted += commit.fetchTotalUpdateRecordsWritten(); totalRecordsWritten += commit.fetchTotalRecordsWritten(); @@ -80,43 +77,32 @@ public class StatsCommand implements CommandMarker { if (totalRecordsUpserted > 0) { waf = df.format((float) totalRecordsWritten / totalRecordsUpserted); } - rows[i] = new String[]{"Total", String.valueOf(totalRecordsUpserted), - String.valueOf(totalRecordsWritten), waf}; - return HoodiePrintHelper.print( - new String[]{"CommitTime", "Total Upserted", "Total Written", - "Write Amplifiation Factor"}, rows); + rows[i] = new String[] {"Total", String.valueOf(totalRecordsUpserted), String.valueOf(totalRecordsWritten), waf}; + return HoodiePrintHelper + .print(new String[] {"CommitTime", "Total Upserted", "Total Written", "Write Amplifiation Factor"}, + rows); } private String[] printFileSizeHistogram(String commitTime, Snapshot s) { - return new String[]{ - commitTime, - NumericUtils.humanReadableByteCount(s.getMin()), - NumericUtils.humanReadableByteCount(s.getValue(0.1)), - NumericUtils.humanReadableByteCount(s.getMedian()), - NumericUtils.humanReadableByteCount(s.getMean()), - NumericUtils.humanReadableByteCount(s.get95thPercentile()), - NumericUtils.humanReadableByteCount(s.getMax()), - String.valueOf(s.size()), - NumericUtils.humanReadableByteCount(s.getStdDev()) - }; + return new String[] {commitTime, NumericUtils.humanReadableByteCount(s.getMin()), + NumericUtils.humanReadableByteCount(s.getValue(0.1)), NumericUtils.humanReadableByteCount(s.getMedian()), + NumericUtils.humanReadableByteCount(s.getMean()), NumericUtils.humanReadableByteCount(s.get95thPercentile()), + NumericUtils.humanReadableByteCount(s.getMax()), String.valueOf(s.size()), + NumericUtils.humanReadableByteCount(s.getStdDev())}; } @CliCommand(value = "stats filesizes", help = "File Sizes. Display summary stats on sizes of files") - public String fileSizeStats( - @CliOption(key = { - "partitionPath"}, help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") - final String globRegex) throws IOException { + public String fileSizeStats(@CliOption(key = { + "partitionPath"}, help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") final + String globRegex) throws IOException { FileSystem fs = HoodieCLI.fs; - String globPath = String.format("%s/%s/*", - HoodieCLI.tableMetadata.getBasePath(), - globRegex); + String globPath = String.format("%s/%s/*", HoodieCLI.tableMetadata.getBasePath(), globRegex); FileStatus[] statuses = fs.globStatus(new Path(globPath)); // max, min, #small files < 10MB, 50th, avg, 95th - final int MAX_FILES = 1000000; Histogram globalHistogram = new Histogram(new UniformReservoir(MAX_FILES)); HashMap commitHistoMap = new HashMap(); for (FileStatus fileStatus : statuses) { @@ -138,8 +124,8 @@ public class StatsCommand implements CommandMarker { Snapshot s = globalHistogram.getSnapshot(); rows[ind++] = printFileSizeHistogram("ALL", s); - return HoodiePrintHelper.print( - new String[]{"CommitTime", "Min", "10th", "50th", "avg", "95th", "Max", "NumFiles", - "StdDev"}, rows); + return HoodiePrintHelper + .print(new String[] {"CommitTime", "Min", "10th", "50th", "avg", "95th", "Max", "NumFiles", "StdDev"}, + rows); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java index 3733a8c0c..676b6ecc6 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java @@ -25,9 +25,7 @@ import org.springframework.stereotype.Component; public class UtilsCommand implements CommandMarker { @CliCommand(value = "utils loadClass", help = "Load a class") - public String loadClass( - @CliOption(key = {"class"}, help = "Check mode") final String clazz - ) throws Exception { + public String loadClass(@CliOption(key = {"class"}, help = "Check mode") final String clazz) throws Exception { Class klass = Class.forName(clazz); return klass.getProtectionDomain().getCodeSource().getLocation().toExternalForm(); } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java index 71ed5aca6..55c2471a9 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java @@ -25,15 +25,12 @@ import java.util.List; public class CommitUtil { - public static long countNewRecords(HoodieTableMetaClient target, List commitsToCatchup) - throws IOException { + public static long countNewRecords(HoodieTableMetaClient target, List commitsToCatchup) throws IOException { long totalNew = 0; - HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline() - .filterCompletedInstants(); + HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants(); for (String commit : commitsToCatchup) { - HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(timeline - .getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)) - .get()); + HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes( + timeline.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)).get()); totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten(); } return totalNew; diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java index 52ec668c2..7e3964349 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java @@ -27,7 +27,7 @@ import org.joda.time.DateTime; public class HiveUtil { - private static String driverName = "org.apache.hive.jdbc.HiveDriver"; + private static final String driverName = "org.apache.hive.jdbc.HiveDriver"; static { try { @@ -39,8 +39,7 @@ public class HiveUtil { private static Connection connection; - private static Connection getConnection(String jdbcUrl, String user, String pass) - throws SQLException { + private static Connection getConnection(String jdbcUrl, String user, String pass) throws SQLException { DataSource ds = getDatasource(jdbcUrl, user, pass); return ds.getConnection(); } @@ -54,8 +53,8 @@ public class HiveUtil { return ds; } - public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String dbName, - String user, String pass) throws SQLException { + public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String dbName, String user, String pass) + throws SQLException { Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass); ResultSet rs = null; Statement stmt = conn.createStatement(); @@ -64,15 +63,13 @@ public class HiveUtil { stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat"); stmt.execute("set hive.stats.autogather=false"); rs = stmt.executeQuery( - "select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + source - .getTableConfig() - .getTableName()); + "select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + + source.getTableConfig().getTableName()); long count = -1; if (rs.next()) { count = rs.getLong("cnt"); } - System.out - .println("Total records in " + source.getTableConfig().getTableName() + " is " + count); + System.out.println("Total records in " + source.getTableConfig().getTableName() + " is " + count); return count; } finally { if (rs != null) { @@ -84,22 +81,19 @@ public class HiveUtil { } } - public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, - int partitions, String user, String pass) throws SQLException { + public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, int partitions, + String user, String pass) throws SQLException { DateTime dateTime = DateTime.now(); - String endDateStr = - dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" + - String.format("%02d", dateTime.getDayOfMonth()); + String endDateStr = dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" + + String.format("%02d", dateTime.getDayOfMonth()); dateTime = dateTime.minusDays(partitions); - String startDateStr = - dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" + - String.format("%02d", dateTime.getDayOfMonth()); + String startDateStr = dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" + + String.format("%02d", dateTime.getDayOfMonth()); System.out.println("Start date " + startDateStr + " and end date " + endDateStr); return countRecords(jdbcUrl, source, srcDb, startDateStr, endDateStr, user, pass); } - private static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, - String startDateStr, + private static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, String startDateStr, String endDateStr, String user, String pass) throws SQLException { Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass); ResultSet rs = null; @@ -109,9 +103,8 @@ public class HiveUtil { stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat"); stmt.execute("set hive.stats.autogather=false"); rs = stmt.executeQuery( - "select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source.getTableConfig() - .getTableName() + " where datestr>'" + startDateStr + "' and datestr<='" - + endDateStr + "'"); + "select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source.getTableConfig().getTableName() + + " where datestr>'" + startDateStr + "' and datestr<='" + endDateStr + "'"); if (rs.next()) { return rs.getLong("cnt"); } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java index 4d926cea3..894bcfae4 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java @@ -24,7 +24,7 @@ import java.util.logging.Logger; public class InputStreamConsumer extends Thread { - protected final static Logger LOG = Logger.getLogger(InputStreamConsumer.class.getName()); + protected static final Logger LOG = Logger.getLogger(InputStreamConsumer.class.getName()); private InputStream is; public InputStreamConsumer(InputStream is) { diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java index d85626d35..74fce58f5 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java @@ -35,13 +35,10 @@ public class SparkUtil { * TODO: Need to fix a bunch of hardcoded stuff here eg: history server, spark distro */ public static SparkLauncher initLauncher(String propertiesFile) throws URISyntaxException { - String currentJar = new File( - SparkUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath()) + String currentJar = new File(SparkUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath()) .getAbsolutePath(); - SparkLauncher sparkLauncher = - new SparkLauncher().setAppResource(currentJar) - .setMainClass(SparkMain.class.getName()) - .setPropertiesFile(propertiesFile); + SparkLauncher sparkLauncher = new SparkLauncher().setAppResource(currentJar).setMainClass(SparkMain.class.getName()) + .setPropertiesFile(propertiesFile); File libDirectory = new File(new File(currentJar).getParent(), "lib"); for (String library : libDirectory.list()) { sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath()); @@ -60,8 +57,7 @@ public class SparkUtil { // Configure hadoop conf sparkConf.set("spark.hadoop.mapred.output.compress", "true"); sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true"); - sparkConf.set("spark.hadoop.mapred.output.compression.codec", - "org.apache.hadoop.io.compress.GzipCodec"); + sparkConf.set("spark.hadoop.mapred.output.compression.codec", "org.apache.hadoop.io.compress.GzipCodec"); sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK"); sparkConf = HoodieWriteClient.registerClasses(sparkConf); 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 82bb9218f..9a2bed3ab 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -50,17 +50,17 @@ import scala.Tuple2; */ public class HoodieReadClient implements Serializable { - private static Logger logger = LogManager.getLogger(HoodieReadClient.class); + private static final Logger logger = LogManager.getLogger(HoodieReadClient.class); - private transient final JavaSparkContext jsc; + private final transient JavaSparkContext jsc; - private transient final FileSystem fs; + private final transient FileSystem fs; /** * TODO: We need to persist the index type into hoodie.properties and be able to access the index * just with a simple basepath pointing to the dataset. Until, then just always assume a * BloomIndex */ - private transient final HoodieIndex index; + private final transient HoodieIndex index; private final HoodieTimeline commitTimeline; private HoodieTable hoodieTable; private transient Optional sqlContextOpt; @@ -69,8 +69,7 @@ public class HoodieReadClient implements Serializ * @param basePath path to Hoodie dataset */ public HoodieReadClient(JavaSparkContext jsc, String basePath) { - this(jsc, HoodieWriteConfig.newBuilder() - .withPath(basePath) + this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath) // by default we use HoodieBloomIndex .withIndexConfig( HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) @@ -78,7 +77,6 @@ public class HoodieReadClient implements Serializ } /** - * * @param jsc * @param basePath * @param sqlContext @@ -96,9 +94,9 @@ public class HoodieReadClient implements Serializ this.jsc = jsc; this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); // Create a Hoodie table which encapsulated the commits and files visible - this.hoodieTable = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), - clientConfig); + this.hoodieTable = HoodieTable + .getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), + clientConfig); this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants(); this.index = HoodieIndex.createIndex(clientConfig, jsc); this.sqlContextOpt = Optional.absent(); @@ -126,33 +124,27 @@ public class HoodieReadClient implements Serializ * * @return a dataframe */ - public Dataset read(JavaRDD hoodieKeys, int parallelism) - throws Exception { + public Dataset read(JavaRDD hoodieKeys, int parallelism) throws Exception { assertSqlContext(); - JavaPairRDD> keyToFileRDD = - index.fetchRecordLocation(hoodieKeys, hoodieTable); - List paths = keyToFileRDD - .filter(keyFileTuple -> keyFileTuple._2().isPresent()) - .map(keyFileTuple -> keyFileTuple._2().get()) - .collect(); + JavaPairRDD> keyToFileRDD = index + .fetchRecordLocation(hoodieKeys, hoodieTable); + List paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent()) + .map(keyFileTuple -> keyFileTuple._2().get()).collect(); // record locations might be same for multiple keys, so need a unique list Set uniquePaths = new HashSet<>(paths); Dataset originalDF = sqlContextOpt.get().read() .parquet(uniquePaths.toArray(new String[uniquePaths.size()])); StructType schema = originalDF.schema(); - JavaPairRDD keyRowRDD = originalDF.javaRDD() - .mapToPair(row -> { - HoodieKey key = new HoodieKey( - row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD), - row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD)); - return new Tuple2<>(key, row); - }); + JavaPairRDD keyRowRDD = originalDF.javaRDD().mapToPair(row -> { + HoodieKey key = new HoodieKey(row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD), + row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD)); + return new Tuple2<>(key, row); + }); // Now, we need to further filter out, for only rows that match the supplied hoodie keys - JavaRDD rowRDD = keyRowRDD.join(keyToFileRDD, parallelism) - .map(tuple -> tuple._2()._1()); + JavaRDD rowRDD = keyRowRDD.join(keyToFileRDD, parallelism).map(tuple -> tuple._2()._1()); return sqlContextOpt.get().createDataFrame(rowRDD, schema); } 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 bb5613f2a..93dd4c292 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -81,18 +81,18 @@ import scala.Tuple2; /** * Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient * mutations on a HDFS dataset [upsert()] - * + *

* Note that, at any given time, there can only be one Spark job performing these operatons on a * Hoodie dataset. */ public class HoodieWriteClient implements Serializable { private static Logger logger = LogManager.getLogger(HoodieWriteClient.class); - private transient final FileSystem fs; - private transient final JavaSparkContext jsc; + private final transient FileSystem fs; + private final transient JavaSparkContext jsc; private final HoodieWriteConfig config; - private transient final HoodieMetrics metrics; - private transient final HoodieIndex index; + private final transient HoodieMetrics metrics; + private final transient HoodieIndex index; private transient Timer.Context writeContext = null; /** @@ -100,8 +100,7 @@ public class HoodieWriteClient implements Seriali * @param clientConfig * @throws Exception */ - public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) - throws Exception { + public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) throws Exception { this(jsc, clientConfig, false); } @@ -129,6 +128,12 @@ public class HoodieWriteClient implements Seriali } } + public static SparkConf registerClasses(SparkConf conf) { + conf.registerKryoClasses( + new Class[] {HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); + return conf; + } + /** * Filter out HoodieRecords that already exists in the output folder. This is useful in * deduplication. @@ -139,8 +144,7 @@ public class HoodieWriteClient implements Seriali 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); JavaRDD> recordsWithLocation = index.tagLocation(hoodieRecords, table); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); @@ -153,9 +157,8 @@ public class HoodieWriteClient implements Seriali HoodieTable table = getTableAndInitCtx(); try { // De-dupe/merge if needed - JavaRDD> dedupedRecords = - combineOnCondition(config.shouldCombineBeforeUpsert(), records, - config.getUpsertShuffleParallelism()); + JavaRDD> dedupedRecords = combineOnCondition( + config.shouldCombineBeforeUpsert(), records, config.getUpsertShuffleParallelism()); // perform index loop up to get existing location of records JavaRDD> taggedRecords = index.tagLocation(dedupedRecords, table); @@ -170,12 +173,12 @@ public class HoodieWriteClient implements Seriali /** * Upserts the given prepared records into the Hoodie table, at the supplied commitTime. - * + *

* This implementation requires that the input records are already tagged, and de-duped if * 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, @@ -187,19 +190,19 @@ public class HoodieWriteClient implements Seriali if (e instanceof HoodieUpsertException) { throw (HoodieUpsertException) e; } - throw new HoodieUpsertException("Failed to upsert prepared records for commit time " + - commitTime, e); + throw new HoodieUpsertException( + "Failed to upsert prepared records for commit time " + commitTime, e); } } /** * Inserts the given HoodieRecords, into the table. This API is intended to be used for normal * writes. - * + *

* 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 */ @@ -207,9 +210,8 @@ public class HoodieWriteClient implements Seriali HoodieTable table = getTableAndInitCtx(); try { // De-dupe/merge if needed - JavaRDD> dedupedRecords = - combineOnCondition(config.shouldCombineBeforeInsert(), records, - config.getInsertShuffleParallelism()); + JavaRDD> dedupedRecords = combineOnCondition( + config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism()); return upsertRecordsInternal(dedupedRecords, commitTime, table, false); } catch (Throwable e) { @@ -222,13 +224,13 @@ public class HoodieWriteClient implements Seriali /** * Inserts the given prepared records into the Hoodie table, at the supplied commitTime. - * + *

* This implementation skips the index check, skips de-duping and is able to leverage benefits * such as small file handling/blocking alignment, as with insert(), by profiling the workload. * The prepared HoodieRecords should be de-duped if needed. * * @param preppedRecords HoodieRecords to insert - * @param commitTime Commit Time handle + * @param commitTime Commit Time handle * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public JavaRDD insertPreppedRecords(JavaRDD> preppedRecords, @@ -240,8 +242,8 @@ public class HoodieWriteClient implements Seriali if (e instanceof HoodieInsertException) { throw e; } - throw new HoodieInsertException("Failed to insert prepared records for commit time " + - commitTime, e); + throw new HoodieInsertException( + "Failed to insert prepared records for commit time " + commitTime, e); } } @@ -249,12 +251,12 @@ public class HoodieWriteClient implements Seriali * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to * Hoodie). - * + *

* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and * 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 */ @@ -267,36 +269,33 @@ public class HoodieWriteClient implements Seriali * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to * Hoodie). - * + *

* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and * attempts to control the numbers of files with less memory compared to the {@link * HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own * 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, + public JavaRDD bulkInsert(JavaRDD> records, final String commitTime, Option bulkInsertPartitioner) { HoodieTable table = getTableAndInitCtx(); try { // De-dupe/merge if needed - JavaRDD> dedupedRecords = - combineOnCondition(config.shouldCombineBeforeInsert(), records, - config.getInsertShuffleParallelism()); + JavaRDD> dedupedRecords = combineOnCondition( + config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism()); return bulkInsertInternal(dedupedRecords, commitTime, table, bulkInsertPartitioner); } catch (Throwable e) { if (e instanceof HoodieInsertException) { throw e; } - throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, - e); + throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, e); } } @@ -304,22 +303,21 @@ public class HoodieWriteClient implements Seriali * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk * loads into a Hoodie table for the very first time (e.g: converting an existing dataset to * Hoodie). The input records should contain no duplicates if needed. - * + *

* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and * attempts to control the numbers of files with less memory compared to the {@link * HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own * 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, - final String commitTime, - Option bulkInsertPartitioner) { + final String commitTime, Option bulkInsertPartitioner) { HoodieTable table = getTableAndInitCtx(); try { return bulkInsertInternal(preppedRecords, commitTime, table, bulkInsertPartitioner); @@ -327,35 +325,29 @@ public class HoodieWriteClient implements Seriali if (e instanceof HoodieInsertException) { throw e; } - throw new HoodieInsertException("Failed to bulk insert prepared records for commit time " + - commitTime, e); + throw new HoodieInsertException( + "Failed to bulk insert prepared records for commit time " + commitTime, e); } } - private JavaRDD bulkInsertInternal( - JavaRDD> dedupedRecords, - String commitTime, - HoodieTable table, + private JavaRDD bulkInsertInternal(JavaRDD> dedupedRecords, + String commitTime, HoodieTable table, Option bulkInsertPartitioner) { final JavaRDD> repartitionedRecords; if (bulkInsertPartitioner.isDefined()) { - repartitionedRecords = - bulkInsertPartitioner.get().repartitionRecords(dedupedRecords, - config.getBulkInsertShuffleParallelism()); + repartitionedRecords = bulkInsertPartitioner.get() + .repartitionRecords(dedupedRecords, config.getBulkInsertShuffleParallelism()); } else { // Now, sort the records and line them up nicely for loading. - repartitionedRecords = dedupedRecords - .sortBy(record -> { - // Let's use "partitionPath + key" as the sort key. Spark, will ensure - // the records split evenly across RDD partitions, such that small partitions fit - // into 1 RDD partition, while big ones spread evenly across multiple RDD partitions - return String - .format("%s+%s", record.getPartitionPath(), record.getRecordKey()); - }, true, config.getBulkInsertShuffleParallelism()); + repartitionedRecords = dedupedRecords.sortBy(record -> { + // Let's use "partitionPath + key" as the sort key. Spark, will ensure + // the records split evenly across RDD partitions, such that small partitions fit + // into 1 RDD partition, while big ones spread evenly across multiple RDD partitions + return String.format("%s+%s", record.getPartitionPath(), record.getRecordKey()); + }, true, config.getBulkInsertShuffleParallelism()); } JavaRDD writeStatusRDD = repartitionedRecords - .mapPartitionsWithIndex(new BulkInsertMapFunction(commitTime, config, table), - true) + .mapPartitionsWithIndex(new BulkInsertMapFunction(commitTime, config, table), true) .flatMap(writeStatuses -> writeStatuses.iterator()); return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime); @@ -375,8 +367,7 @@ public class HoodieWriteClient implements Seriali } private JavaRDD> combineOnCondition(boolean condition, - JavaRDD> records, - int parallelism) { + JavaRDD> records, int parallelism) { if (condition) { return deduplicateRecords(records, parallelism); } @@ -390,8 +381,8 @@ public class HoodieWriteClient implements Seriali * files) are rolled back based on commit time. // TODO : Create a new WorkloadProfile metadata * file instead of using HoodieCommitMetadata */ - private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, - HoodieTable table, String commitTime) throws HoodieCommitException { + private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, HoodieTable table, + String commitTime) throws HoodieCommitException { try { HoodieCommitMetadata metadata = new HoodieCommitMetadata(); profile.getPartitionPaths().stream().forEach(path -> { @@ -416,9 +407,7 @@ public class HoodieWriteClient implements Seriali } private JavaRDD upsertRecordsInternal(JavaRDD> preppedRecords, - String commitTime, - HoodieTable hoodieTable, - final boolean isUpsert) { + String commitTime, HoodieTable hoodieTable, final boolean isUpsert) { // Cache the tagged records, so we don't end up computing both // TODO: Consistent contract in HoodieWriteClient regarding preppedRecord storage level handling @@ -441,20 +430,16 @@ public class HoodieWriteClient implements Seriali JavaRDD writeStatusRDD = partitionedRecords .mapPartitionsWithIndex((partition, recordItr) -> { if (isUpsert) { - return hoodieTable - .handleUpsertPartition(commitTime, partition, recordItr, partitioner); + return hoodieTable.handleUpsertPartition(commitTime, partition, recordItr, partitioner); } else { - return hoodieTable - .handleInsertPartition(commitTime, partition, recordItr, partitioner); + return hoodieTable.handleInsertPartition(commitTime, partition, recordItr, partitioner); } - }, true) - .flatMap(writeStatuses -> writeStatuses.iterator()); + }, true).flatMap(writeStatuses -> writeStatuses.iterator()); return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime); } - private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, - WorkloadProfile profile) { + private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, WorkloadProfile profile) { if (isUpsert) { return table.getUpsertPartitioner(profile); } else { @@ -474,13 +459,9 @@ public class HoodieWriteClient implements Seriali private JavaRDD> partition(JavaRDD> dedupedRecords, Partitioner partitioner) { - return dedupedRecords - .mapToPair(record -> - new Tuple2<>( - new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), - record)) - .partitionBy(partitioner) - .map(tuple -> tuple._2()); + return dedupedRecords.mapToPair(record -> new Tuple2<>( + new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record)) + .partitionBy(partitioner).map(tuple -> tuple._2()); } /** @@ -493,12 +474,10 @@ public class HoodieWriteClient implements Seriali /** * Commit changes performed at the given commitTime marker */ - 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); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); return commit(commitTime, writeStatuses, extraMetadata, table.getCommitActionType()); } @@ -508,15 +487,13 @@ public class HoodieWriteClient implements Seriali 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); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - List> stats = writeStatuses - .mapToPair((PairFunction) writeStatus -> - new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat())) - .collect(); + List> stats = writeStatuses.mapToPair( + (PairFunction) writeStatus -> new Tuple2<>( + writeStatus.getPartitionPath(), writeStatus.getStat())).collect(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); for (Tuple2 stat : stats) { @@ -529,10 +506,9 @@ public class HoodieWriteClient implements Seriali if (finalizeCtx != null && result.isPresent()) { Optional durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop())); durationInMs.ifPresent(duration -> { - logger.info("Finalize write elapsed time (milliseconds): " + duration); - metrics.updateFinalizeWriteMetrics(duration, result.get()); - } - ); + logger.info("Finalize write elapsed time (milliseconds): " + duration); + metrics.updateFinalizeWriteMetrics(duration, result.get()); + }); } // add in extra metadata @@ -541,8 +517,7 @@ public class HoodieWriteClient implements Seriali } try { - activeTimeline.saveAsComplete( - new HoodieInstant(true, actionType, commitTime), + activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, commitTime), Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); // Save was a success // Do a inline compaction if enabled @@ -566,9 +541,9 @@ public class HoodieWriteClient implements Seriali } if (writeContext != null) { long durationInMs = metrics.getDurationInMs(writeContext.stop()); - metrics.updateCommitMetrics( - HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), durationInMs, - metadata); + metrics + .updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), + durationInMs, metadata); writeContext = null; } logger.info("Committed " + commitTime); @@ -587,20 +562,19 @@ public class HoodieWriteClient implements Seriali * Savepoint a specific commit. Latest version of data files as of the passed in commitTime will * be referenced in the savepoint and will never be cleaned. The savepointed commit will never be * rolledback or archived. - * + *

* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be * manually created and deleted. - * + *

* 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); if (table.getCompletedCommitTimeline().empty()) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } @@ -614,21 +588,20 @@ public class HoodieWriteClient implements Seriali * Savepoint a specific commit. Latest version of data files as of the passed in commitTime will * be referenced in the savepoint and will never be cleaned. The savepointed commit will never be * rolledback or archived. - * + *

* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be * manually created and deleted. - * + *

* 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); Optional cleanInstant = table.getCompletedCleanTimeline().lastInstant(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, @@ -646,8 +619,7 @@ public class HoodieWriteClient implements Seriali table.getActiveTimeline().getInstantDetails(cleanInstant.get()).get()); lastCommitRetained = cleanMetadata.getEarliestCommitToRetain(); } else { - lastCommitRetained = - table.getCompletedCommitTimeline().firstInstant().get().getTimestamp(); + lastCommitRetained = table.getCompletedCommitTimeline().firstInstant().get().getTimestamp(); } // Cannot allow savepoint time on a commit that could have been cleaned @@ -656,25 +628,24 @@ public class HoodieWriteClient implements Seriali "Could not savepoint commit " + commitTime + " as this is beyond the lookup window " + lastCommitRetained); - Map> latestFilesMap = jsc.parallelize( - FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(), + Map> latestFilesMap = jsc.parallelize(FSUtils + .getAllPartitionPaths(fs, table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) .mapToPair((PairFunction>) partitionPath -> { // Scan all partitions files with this commit time logger.info("Collecting latest files in partition path " + partitionPath); TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView(); - List latestFiles = - view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime) - .map(HoodieDataFile::getFileName).collect(Collectors.toList()); + List latestFiles = view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime) + .map(HoodieDataFile::getFileName).collect(Collectors.toList()); return new Tuple2<>(partitionPath, latestFiles); }).collectAsMap(); - HoodieSavepointMetadata metadata = - AvroUtils.convertSavepointMetadata(user, comment, latestFilesMap); + HoodieSavepointMetadata metadata = AvroUtils + .convertSavepointMetadata(user, comment, latestFilesMap); // Nothing to save in the savepoint - table.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime), - AvroUtils.serializeSavepointMetadata(metadata)); + table.getActiveTimeline() + .saveAsComplete(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime), + AvroUtils.serializeSavepointMetadata(metadata)); logger.info("Savepoint " + commitTime + " created"); return true; } catch (IOException e) { @@ -691,22 +662,20 @@ public class HoodieWriteClient implements Seriali */ 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); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieInstant savePoint = - new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); - boolean isSavepointPresent = - table.getCompletedSavepointTimeline().containsInstant(savePoint); + HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, + savepointTime); + boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint); if (!isSavepointPresent) { logger.warn("No savepoint present " + savepointTime); return; } activeTimeline.revertToInflight(savePoint); - activeTimeline.deleteInflight( - new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, savepointTime)); + activeTimeline + .deleteInflight(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, savepointTime)); logger.info("Savepoint " + savepointTime + " deleted"); } @@ -719,30 +688,27 @@ public class HoodieWriteClient implements Seriali */ 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); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieTimeline commitTimeline = table.getCommitsTimeline(); - HoodieInstant savePoint = - new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); - boolean isSavepointPresent = - table.getCompletedSavepointTimeline().containsInstant(savePoint); + HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, + savepointTime); + boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint); if (!isSavepointPresent) { throw new HoodieRollbackException("No savepoint for commitTime " + savepointTime); } - List commitsToRollback = - commitTimeline.findInstantsAfter(savepointTime, Integer.MAX_VALUE).getInstants() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + List commitsToRollback = commitTimeline + .findInstantsAfter(savepointTime, Integer.MAX_VALUE).getInstants() + .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); logger.info("Rolling back commits " + commitsToRollback); rollback(commitsToRollback); // Make sure the rollback was successful - Optional lastInstant = - activeTimeline.reload().getCommitsTimeline().filterCompletedInstants() - .lastInstant(); + Optional lastInstant = activeTimeline.reload().getCommitsTimeline() + .filterCompletedInstants().lastInstant(); Preconditions.checkArgument(lastInstant.isPresent()); Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime), savepointTime + "is not the last commit after rolling back " + commitsToRollback @@ -771,16 +737,14 @@ public class HoodieWriteClient implements Seriali // 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); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieTimeline inflightTimeline = table.getInflightCommitTimeline(); HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); // Check if any of the commits is a savepoint - do not allow rollback on those commits - List savepoints = - table.getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp) - .collect(Collectors.toList()); + List savepoints = table.getCompletedSavepointTimeline().getInstants() + .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); commits.forEach(s -> { if (savepoints.contains(s)) { throw new HoodieRollbackException( @@ -800,16 +764,15 @@ public class HoodieWriteClient implements Seriali String lastCommit = commits.get(commits.size() - 1); if (!commitTimeline.empty() && !commitTimeline .findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) { - throw new HoodieRollbackException("Found commits after time :" + lastCommit + - ", please rollback greater commits first"); + throw new HoodieRollbackException( + "Found commits after time :" + lastCommit + ", please rollback greater commits first"); } List inflights = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); if (!inflights.isEmpty() && inflights.indexOf(lastCommit) != inflights.size() - 1) { - throw new HoodieRollbackException( - "Found in-flight commits after time :" + lastCommit + - ", please rollback greater commits first"); + throw new HoodieRollbackException("Found in-flight commits after time :" + lastCommit + + ", please rollback greater commits first"); } List stats = table.rollback(jsc, commits); @@ -817,8 +780,7 @@ public class HoodieWriteClient implements Seriali // cleanup index entries commits.stream().forEach(s -> { if (!index.rollbackCommit(s)) { - throw new HoodieRollbackException( - "Rollback index changes failed, for time :" + s); + throw new HoodieRollbackException("Rollback index changes failed, for time :" + s); } }); logger.info("Index rolled back for commits " + commits); @@ -826,13 +788,12 @@ public class HoodieWriteClient implements Seriali Optional durationInMs = Optional.empty(); if (context != null) { durationInMs = Optional.of(metrics.getDurationInMs(context.stop())); - Long numFilesDeleted = stats.stream() - .mapToLong(stat -> stat.getSuccessDeleteFiles().size()) + Long numFilesDeleted = stats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size()) .sum(); metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted); } - HoodieRollbackMetadata rollbackMetadata = - AvroUtils.convertRollbackMetadata(startRollbackTime, durationInMs, commits, stats); + HoodieRollbackMetadata rollbackMetadata = AvroUtils + .convertRollbackMetadata(startRollbackTime, durationInMs, commits, stats); table.getActiveTimeline().saveAsComplete( new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime), AvroUtils.serializeRollbackMetadata(rollbackMetadata)); @@ -846,8 +807,8 @@ public class HoodieWriteClient implements Seriali table.getActiveTimeline().getRollbackTimeline().getInstants()); } } catch (IOException e) { - throw new HoodieRollbackException("Failed to rollback " + - config.getBasePath() + " commits " + commits, e); + throw new HoodieRollbackException( + "Failed to rollback " + config.getBasePath() + " commits " + commits, e); } } @@ -880,8 +841,7 @@ public class HoodieWriteClient implements Seriali // 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); List cleanStats = table.clean(jsc); if (cleanStats.isEmpty()) { @@ -896,15 +856,15 @@ public class HoodieWriteClient implements Seriali } // Create the metadata and save it - HoodieCleanMetadata metadata = - AvroUtils.convertCleanMetadata(startCleanTime, durationInMs, cleanStats); + HoodieCleanMetadata metadata = AvroUtils + .convertCleanMetadata(startCleanTime, durationInMs, cleanStats); logger.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"); - metrics.updateCleanMetrics(durationInMs.orElseGet(() -> -1L), - metadata.getTotalFilesDeleted()); + metrics + .updateCleanMetrics(durationInMs.orElseGet(() -> -1L), metadata.getTotalFilesDeleted()); - table.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, startCleanTime), - AvroUtils.serializeCleanMetadata(metadata)); + table.getActiveTimeline() + .saveAsComplete(new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, startCleanTime), + AvroUtils.serializeCleanMetadata(metadata)); logger.info("Marked clean started on " + startCleanTime + " as complete"); if (!table.getActiveTimeline().getCleanerTimeline().empty()) { @@ -930,12 +890,10 @@ public class HoodieWriteClient implements Seriali 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); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); String commitActionType = table.getCommitActionType(); - activeTimeline.createInflight( - new HoodieInstant(true, commitActionType, commitTime)); + activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); } /** @@ -948,17 +906,16 @@ public class HoodieWriteClient implements Seriali return commitTime; } - /** Since MOR tableType default to {@link HoodieTimeline#DELTA_COMMIT_ACTION}, - * we need to explicitly set to {@link HoodieTimeline#COMMIT_ACTION} for compaction + /** + * Since MOR tableType default to {@link HoodieTimeline#DELTA_COMMIT_ACTION}, we need to + * explicitly set to {@link HoodieTimeline#COMMIT_ACTION} for compaction */ 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); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - String commitActionType = HoodieTimeline.COMMIT_ACTION; - activeTimeline.createInflight( - new HoodieInstant(true, commitActionType, commitTime)); + String commitActionType = HoodieTimeline.COMMIT_ACTION; + activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime)); } /** @@ -968,8 +925,7 @@ public class HoodieWriteClient implements Seriali 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); JavaRDD statuses = table.compact(jsc, commitTime); // Trigger the insert and collect statuses statuses = statuses.persist(config.getWriteStatusStorageLevel()); @@ -980,9 +936,6 @@ public class HoodieWriteClient implements Seriali /** * Commit a compaction operation - * @param commitTime - * @param writeStatuses - * @param extraMetadata */ public void commitCompaction(String commitTime, JavaRDD writeStatuses, Optional> extraMetadata) { @@ -992,8 +945,6 @@ public class HoodieWriteClient implements Seriali /** * Commit a compaction operation - * @param commitTime - * @param writeStatuses */ public void commitCompaction(String commitTime, JavaRDD writeStatuses) { String commitCompactionActionType = HoodieActiveTimeline.COMMIT_ACTION; @@ -1006,8 +957,8 @@ public class HoodieWriteClient implements Seriali */ 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); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), + config.getBasePath(), true); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); JavaRDD compactedStatuses = table.compact(jsc, compactionCommitTime); if (!compactedStatuses.isEmpty()) { @@ -1029,8 +980,7 @@ public class HoodieWriteClient implements Seriali } private void commitForceCompaction(JavaRDD writeStatuses, - HoodieTableMetaClient metaClient, - String compactionCommitTime) { + HoodieTableMetaClient metaClient, String compactionCommitTime) { List updateStatusMap = writeStatuses.map(writeStatus -> writeStatus.getStat()) .collect(); @@ -1054,12 +1004,6 @@ public class HoodieWriteClient implements Seriali } } - public static SparkConf registerClasses(SparkConf conf) { - conf.registerKryoClasses( - new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class}); - return conf; - } - /** * Deduplicate Hoodie records, using the given deduplication funciton. */ @@ -1074,13 +1018,13 @@ public class HoodieWriteClient implements Seriali return new Tuple2<>(key, record); }) .reduceByKey((rec1, rec2) -> { - @SuppressWarnings("unchecked") - T reducedData = (T) rec1.getData().preCombine(rec2.getData()); - // we cannot allow the user to change the key or partitionPath, since that will affect everything + @SuppressWarnings("unchecked") T reducedData = (T) rec1.getData() + .preCombine(rec2.getData()); + // we cannot allow the user to change the key or partitionPath, since that will affect + // everything // so pick it from one of the records. return new HoodieRecord(rec1.getKey(), reducedData); - }, parallelism) - .map(recordTuple -> recordTuple._2()); + }, parallelism).map(recordTuple -> recordTuple._2()); } /** @@ -1088,8 +1032,7 @@ public class HoodieWriteClient implements Seriali */ private void rollbackInflightCommits() { HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); List commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); @@ -1103,7 +1046,6 @@ public class HoodieWriteClient implements Seriali writeContext = metrics.getCommitCtx(); // Create a Hoodie table which encapsulated the commits and files visible return HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); } } 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 7854e128e..c28db2567 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 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 markFailure(HoodieRecord record, Throwable t, Optional> optionalRecordMetadata) { @@ -106,14 +106,14 @@ public class WriteStatus implements Serializable { return globalError != null; } - public void setGlobalError(Throwable t) { - this.globalError = t; - } - public Throwable getGlobalError() { return this.globalError; } + public void setGlobalError(Throwable t) { + this.globalError = t; + } + public List getWrittenRecords() { return writtenRecords; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java index 948a1e00e..515434a27 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java @@ -30,10 +30,6 @@ public class DefaultHoodieConfig implements Serializable { this.props = props; } - public Properties getProps() { - return props; - } - public static void setDefaultOnCondition(Properties props, boolean condition, String propName, String defaultValue) { if (condition) { @@ -48,4 +44,8 @@ public class DefaultHoodieConfig implements Serializable { } } + public Properties getProps() { + return props; + } + } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java index dbb47e6bd..3d10d1b73 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieCompactionConfig.java @@ -34,81 +34,76 @@ import javax.annotation.concurrent.Immutable; public class HoodieCompactionConfig extends DefaultHoodieConfig { public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy"; - private static final String DEFAULT_CLEANER_POLICY = - HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name(); - public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic"; - private static final String DEFAULT_AUTO_CLEAN = "true"; - // Turn on inline compaction - after fw delta commits a inline compaction will be run public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline"; - private static final String DEFAULT_INLINE_COMPACT = "false"; - // Run a compaction every N delta commits - public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits"; - private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10"; - + public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = + "hoodie.compact.inline.max" + ".delta.commits"; public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = - "hoodie.cleaner.fileversions.retained"; - private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3"; - + "hoodie.cleaner.fileversions" + ".retained"; public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained"; - private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24"; - public static final String MAX_COMMITS_TO_KEEP = "hoodie.keep.max.commits"; - private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128); public static final String MIN_COMMITS_TO_KEEP = "hoodie.keep.min.commits"; - private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96); // Upsert uses this file size to compact new data onto existing files.. public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit"; // Turned off by default public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(0); - /** * Configs related to specific table types **/ // Number of inserts, that will be put each partition/bucket for writing - public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = "hoodie.copyonwrite.insert.split.size"; + public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = + "hoodie.copyonwrite.insert" + ".split.size"; // The rationale to pick the insert parallelism is the following. Writing out 100MB files, // with atleast 1kb records, means 100K records per file. we just overprovision to 500K public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000); - - // Config to control whether we control insert split sizes automatically based on average record sizes - public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert.auto.split"; + // Config to control whether we control insert split sizes automatically based on average + // record sizes + public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = + "hoodie.copyonwrite.insert" + ".auto.split"; // its off by default public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(false); - - - // This value is used as a guessimate for the record size, if we can't determine this from previous commits - public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = "hoodie.copyonwrite.record.size.estimate"; - // Used to determine how much more can be packed into a small file, before it exceeds the size limit. + // This value is used as a guessimate for the record size, if we can't determine this from + // previous commits + public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = + "hoodie.copyonwrite" + ".record.size.estimate"; + // Used to determine how much more can be packed into a small file, before it exceeds the size + // limit. public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String .valueOf(1024); - public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism"; public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200); - public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io"; // 500GB of target IO per compaction (both read and write) public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024); - public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy"; // 200GB of target IO per compaction public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class .getName(); - // used to merge records written to log file public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName(); public static final String PAYLOAD_CLASS_PROP = "hoodie.compaction.payload.class"; // used to choose a trade off between IO vs Memory when performing compaction process - // Depending on outputfile_size and memory provided, choose true to avoid OOM for large file size + small memory - public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "hoodie.compaction.lazy.block.read"; + // Depending on outputfile_size and memory provided, choose true to avoid OOM for large file + // size + small memory + public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = + "hoodie.compaction.lazy" + ".block.read"; public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "false"; - // used to choose whether to enable reverse log reading (reverse log traversal) - public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = "hoodie.compaction.reverse.log.read"; + public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = + "hoodie.compaction" + ".reverse.log.read"; public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false"; + private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS + .name(); + private static final String DEFAULT_AUTO_CLEAN = "true"; + private static final String DEFAULT_INLINE_COMPACT = "false"; + private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10"; + private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3"; + private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24"; + private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128); + private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96); private HoodieCompactionConfig(Properties props) { super(props); @@ -159,8 +154,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { } public Builder retainFileVersions(int fileVersionsRetained) { - props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP, - String.valueOf(fileVersionsRetained)); + props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP, String.valueOf(fileVersionsRetained)); return this; } @@ -238,22 +232,22 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { public HoodieCompactionConfig build() { HoodieCompactionConfig config = new HoodieCompactionConfig(props); - setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), - AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN); - setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), - INLINE_COMPACT_PROP, DEFAULT_INLINE_COMPACT); + setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP, + DEFAULT_AUTO_CLEAN); + setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), INLINE_COMPACT_PROP, + DEFAULT_INLINE_COMPACT); setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP), INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS); - setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP), - CLEANER_POLICY_PROP, DEFAULT_CLEANER_POLICY); + setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP), CLEANER_POLICY_PROP, + DEFAULT_CLEANER_POLICY); setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP), CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED); setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP), CLEANER_COMMITS_RETAINED_PROP, DEFAULT_CLEANER_COMMITS_RETAINED); - setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP), - MAX_COMMITS_TO_KEEP, DEFAULT_MAX_COMMITS_TO_KEEP); - setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP), - MIN_COMMITS_TO_KEEP, DEFAULT_MIN_COMMITS_TO_KEEP); + setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP), MAX_COMMITS_TO_KEEP, + DEFAULT_MAX_COMMITS_TO_KEEP); + setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP), MIN_COMMITS_TO_KEEP, + DEFAULT_MIN_COMMITS_TO_KEEP); setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES), PARQUET_SMALL_FILE_LIMIT_BYTES, DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES); setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE), @@ -263,8 +257,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE), COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE); - setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM), - CLEANER_PARALLELISM, DEFAULT_CLEANER_PARALLELISM); + setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM), CLEANER_PARALLELISM, + DEFAULT_CLEANER_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP), COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY); setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP), @@ -277,9 +271,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig { COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED); HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP)); - Preconditions.checkArgument( - Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer - .parseInt(props.getProperty(MIN_COMMITS_TO_KEEP))); + Preconditions.checkArgument(Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer + .parseInt(props.getProperty(MIN_COMMITS_TO_KEEP))); return config; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java index 9a9687a3d..5a24cc12f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieIndexConfig.java @@ -40,23 +40,25 @@ public class HoodieIndexConfig extends DefaultHoodieConfig { public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism"; // Disable explicit bloom index parallelism setting by default - hoodie auto computes public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0"; - public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = "hoodie.bloom.index.prune.by.ranges"; + public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = + "hoodie.bloom.index.prune.by" + ".ranges"; public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true"; public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching"; public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true"; - public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = "hoodie.bloom.index.input.storage.level"; + public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = + "hoodie.bloom.index.input.storage" + ".level"; public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER"; // ***** HBase Index Configs ***** - public final static String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum"; - public final static String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport"; - public final static String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table"; - public final static String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size"; - public final static String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size"; - public final static String DEFAULT_HBASE_BATCH_SIZE = "100"; + public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum"; + public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport"; + public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table"; + public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size"; + public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size"; + public static final String DEFAULT_HBASE_BATCH_SIZE = "100"; // ***** Bucketed Index Configs ***** - public final static String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets"; + public static final String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets"; private HoodieIndexConfig(Properties props) { super(props); @@ -152,12 +154,12 @@ public class HoodieIndexConfig extends DefaultHoodieConfig { public HoodieIndexConfig build() { HoodieIndexConfig config = new HoodieIndexConfig(props); - setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), - INDEX_TYPE_PROP, DEFAULT_INDEX_TYPE); + setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP, + DEFAULT_INDEX_TYPE); setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES), BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES); - setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), - BLOOM_FILTER_FPP, DEFAULT_BLOOM_FILTER_FPP); + setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), BLOOM_FILTER_FPP, + DEFAULT_BLOOM_FILTER_FPP); setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP), BLOOM_INDEX_PARALLELISM_PROP, DEFAULT_BLOOM_INDEX_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP), diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java index 65f64e41b..aa5f28cc7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMemoryConfig.java @@ -30,8 +30,8 @@ import org.apache.spark.util.Utils; @Immutable public class HoodieMemoryConfig extends DefaultHoodieConfig { - // This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use during merge - // This makes it easier to scale this value as one increases the spark.executor.memory + // This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use + // during merge. This makes it easier to scale this value as one increases the spark.executor.memory public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = "hoodie.memory.merge.fraction"; // Default max memory fraction during hash-merge, excess spills to disk public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE = String.valueOf(0.6); @@ -87,19 +87,21 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig { } /** - * Dynamic calculation of max memory to use for for spillable map. user.available.memory = - * spark.executor.memory * (1 - spark.memory.fraction) spillable.available.memory = - * user.available.memory * hoodie.memory.fraction. Anytime the spark.executor.memory or the - * spark.memory.fraction is changed, the memory used for spillable map changes accordingly + * Dynamic calculation of max memory to use for for spillable map. user.available.memory = spark.executor.memory * + * (1 - spark.memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime + * the spark.executor.memory or the spark.memory.fraction is changed, the memory used for spillable map changes + * accordingly */ private long getMaxMemoryAllowedForMerge(String maxMemoryFraction) { final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory"; final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction"; - // This is hard-coded in spark code {@link https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala#L231} - // so have to re-define this here + // This is hard-coded in spark code {@link + // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/ + // spark/memory/UnifiedMemoryManager.scala#L231} so have to re-define this here final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6"; - // This is hard-coded in spark code {@link https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/spark/SparkContext.scala#L471} - // so have to re-define this here + // This is hard-coded in spark code {@link + // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/ + // spark/SparkContext.scala#L471} so have to re-define this here final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB if (SparkEnv.get() != null) { @@ -109,7 +111,8 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig { DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024 * 1024L); // 0.6 is the default value used by Spark, - // look at {@link https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507} + // look at {@link + // https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507} double memoryFraction = Double .valueOf(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION)); @@ -143,5 +146,4 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig { return config; } } - } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMetricsConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMetricsConfig.java index f3ef71077..3b06e61bd 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMetricsConfig.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieMetricsConfig.java @@ -29,22 +29,22 @@ import javax.annotation.concurrent.Immutable; @Immutable public class HoodieMetricsConfig extends DefaultHoodieConfig { - public final static String METRIC_PREFIX = "hoodie.metrics"; - public final static String METRICS_ON = METRIC_PREFIX + ".on"; - public final static boolean DEFAULT_METRICS_ON = false; - public final static String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type"; - public final static MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = - MetricsReporterType.GRAPHITE; + public static final String METRIC_PREFIX = "hoodie.metrics"; + public static final String METRICS_ON = METRIC_PREFIX + ".on"; + public static final boolean DEFAULT_METRICS_ON = false; + public static final String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type"; + public static final MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = MetricsReporterType + .GRAPHITE; // Graphite - public final static String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite"; - public final static String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host"; - public final static String DEFAULT_GRAPHITE_SERVER_HOST = "localhost"; + public static final String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite"; + public static final String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host"; + public static final String DEFAULT_GRAPHITE_SERVER_HOST = "localhost"; - public final static String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port"; - public final static int DEFAULT_GRAPHITE_SERVER_PORT = 4756; + public static final String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port"; + public static final int DEFAULT_GRAPHITE_SERVER_PORT = 4756; - public final static String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix"; + public static final String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix"; private HoodieMetricsConfig(Properties props) { super(props); @@ -103,14 +103,14 @@ public class HoodieMetricsConfig extends DefaultHoodieConfig { HoodieMetricsConfig config = new HoodieMetricsConfig(props); setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON, String.valueOf(DEFAULT_METRICS_ON)); - setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE), - METRICS_REPORTER_TYPE, DEFAULT_METRICS_REPORTER_TYPE.name()); - setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST), - GRAPHITE_SERVER_HOST, DEFAULT_GRAPHITE_SERVER_HOST); - setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), - GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT)); - setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), - GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT)); + setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE), METRICS_REPORTER_TYPE, + DEFAULT_METRICS_REPORTER_TYPE.name()); + setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST), GRAPHITE_SERVER_HOST, + DEFAULT_GRAPHITE_SERVER_HOST); + setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT, + String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT)); + setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT, + String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT)); return config; } } 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 0eb0a5c0e..b72e6f743 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 @@ -36,10 +36,11 @@ public class HoodieStorageConfig extends DefaultHoodieConfig { public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024); // used to size log files public static final String LOGFILE_SIZE_MAX_BYTES = "hoodie.logfile.max.size"; - public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String.valueOf(1024*1024*1024); // 1 GB + public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String + .valueOf(1024 * 1024 * 1024); // 1 GB // used to size data blocks in log file public static final String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = "hoodie.logfile.data.block.max.size"; - public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256*1024*1024); // 256 MB + public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256 * 1024 * 1024); // 256 MB 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); 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 14909bffa..683ec952d 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 @@ -16,7 +16,6 @@ package com.uber.hoodie.config; - import com.google.common.base.Preconditions; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieCleaningPolicy; @@ -24,15 +23,14 @@ import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.io.compact.strategy.CompactionStrategy; import com.uber.hoodie.metrics.MetricsReporterType; -import org.apache.spark.storage.StorageLevel; - -import javax.annotation.concurrent.Immutable; import java.io.File; import java.io.FileReader; import java.io.IOException; import java.io.InputStream; import java.util.Map; import java.util.Properties; +import javax.annotation.concurrent.Immutable; +import org.apache.spark.storage.StorageLevel; /** * Class storing configs for the {@link com.uber.hoodie.HoodieWriteClient} @@ -40,9 +38,9 @@ import java.util.Properties; @Immutable public class HoodieWriteConfig extends DefaultHoodieConfig { + public static final String TABLE_NAME = "hoodie.table.name"; private static final String BASE_PATH_PROP = "hoodie.base.path"; private static final String AVRO_SCHEMA = "hoodie.avro.schema"; - public static final String TABLE_NAME = "hoodie.table.name"; private static final String DEFAULT_PARALLELISM = "200"; private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism"; private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism"; @@ -57,13 +55,16 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER"; private static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit"; private static final String DEFAULT_HOODIE_AUTO_COMMIT = "true"; - private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning"; + private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = + "hoodie.assume.date" + ".partitioning"; private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false"; private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class"; private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName(); - private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = "hoodie.copyonwrite.use.temp.folder.for.create"; + private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = + "hoodie.copyonwrite.use" + ".temp.folder.for.create"; private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = "false"; - private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "hoodie.copyonwrite.use.temp.folder.for.merge"; + private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = + "hoodie.copyonwrite.use" + ".temp.folder.for.merge"; private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "false"; private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism"; private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM; @@ -72,6 +73,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { super(props); } + public static HoodieWriteConfig.Builder newBuilder() { + return new Builder(); + } + /** * base properties **/ @@ -137,8 +142,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { } public boolean shouldUseTempFolderForCopyOnWrite() { - return shouldUseTempFolderForCopyOnWriteForCreate() || - shouldUseTempFolderForCopyOnWriteForMerge(); + return shouldUseTempFolderForCopyOnWriteForCreate() + || shouldUseTempFolderForCopyOnWriteForMerge(); } public int getFinalizeWriteParallelism() { @@ -154,8 +159,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { } public int getCleanerFileVersionsRetained() { - return Integer.parseInt( - props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP)); + return Integer + .parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP)); } public int getCleanerCommitsRetained() { @@ -177,8 +182,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { } public int getCopyOnWriteInsertSplitSize() { - return Integer.parseInt( - props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE)); + return Integer + .parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE)); } public int getCopyOnWriteRecordSizeEstimate() { @@ -204,8 +209,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { } public int getInlineCompactDeltaCommitMax() { - return Integer.parseInt( - props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP)); + return Integer + .parseInt(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP)); } public CompactionStrategy getCompactionStrategy() { @@ -340,11 +345,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public String getGraphiteMetricPrefix() { return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX); } - - public static HoodieWriteConfig.Builder newBuilder() { - return new Builder(); - } - + /** * memory configs */ @@ -486,15 +487,15 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public Builder withUseTempFolderCopyOnWriteForCreate( boolean shouldUseTempFolderCopyOnWriteForCreate) { - props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, String.valueOf - (shouldUseTempFolderCopyOnWriteForCreate)); + props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, + String.valueOf(shouldUseTempFolderCopyOnWriteForCreate)); return this; } public Builder withUseTempFolderCopyOnWriteForMerge( boolean shouldUseTempFolderCopyOnWriteForMerge) { - props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, String.valueOf - (shouldUseTempFolderCopyOnWriteForMerge)); + props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, + String.valueOf(shouldUseTempFolderCopyOnWriteForMerge)); return this; } @@ -510,8 +511,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM, DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM), - BULKINSERT_PARALLELISM, - DEFAULT_PARALLELISM); + BULKINSERT_PARALLELISM, DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM, DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP), diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java index 08cea4dc7..e69c09b1a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/BufferedIterator.java @@ -21,12 +21,6 @@ import com.google.common.base.Preconditions; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.exception.HoodieException; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.util.SizeEstimator; - import java.util.Iterator; import java.util.Optional; import java.util.concurrent.LinkedBlockingQueue; @@ -35,53 +29,63 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.util.SizeEstimator; /** - * Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It internally samples - * every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in buffer accordingly. This is done to - * ensure that we don't OOM. + * Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It + * internally samples every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in + * buffer accordingly. This is done to ensure that we don't OOM. */ -public class BufferedIterator> - implements Iterator> { +public class BufferedIterator> implements + Iterator> { - private static Logger logger = LogManager.getLogger(BufferedIterator.class); // interval used for polling records in the queue. public static final int RECORD_POLL_INTERVAL_SEC = 5; // rate used for sampling records to determine avg record size in bytes. public static final int RECORD_SAMPLING_RATE = 64; // maximum records that will be cached private static final int RECORD_CACHING_LIMIT = 128 * 1024; - // It indicates number of records to cache. We will be using sampled record's average size to determine how many + private static Logger logger = LogManager.getLogger(BufferedIterator.class); + // It indicates number of records to cache. We will be using sampled record's average size to + // determine how many // records we should cache and will change (increase/decrease) permits accordingly. @VisibleForTesting public final Semaphore rateLimiter = new Semaphore(1); // used for sampling records with "RECORD_SAMPLING_RATE" frequency. public final AtomicLong samplingRecordCounter = new AtomicLong(-1); - // indicates rate limit (number of records to cache). it is updated whenever there is a change in avg record size. - @VisibleForTesting - public int currentRateLimit = 1; // internal buffer to cache buffered records. - private final LinkedBlockingQueue>> buffer = new LinkedBlockingQueue<>(); + private final LinkedBlockingQueue>> buffer = new + LinkedBlockingQueue<>(); // maximum amount of memory to be used for buffering records. private final long bufferMemoryLimit; + // original iterator from where records are read for buffering. + private final Iterator inputIterator; + // it holds the root cause of the exception in case either buffering records (reading from + // inputIterator) fails or + // thread reading records from buffer fails. + private final AtomicReference hasFailed = new AtomicReference(null); + // used for indicating that all the records from buffer are read successfully. + private final AtomicBoolean isDone = new AtomicBoolean(false); + // schema used for fetching insertValue from HoodieRecord. + private final Schema schema; + // indicates rate limit (number of records to cache). it is updated whenever there is a change + // in avg record size. + @VisibleForTesting + public int currentRateLimit = 1; // indicates avg record size in bytes. It is updated whenever a new record is sampled. @VisibleForTesting public long avgRecordSizeInBytes = 0; // indicates number of samples collected so far. private long numSamples = 0; - // original iterator from where records are read for buffering. - private final Iterator inputIterator; - // it holds the root cause of the exception in case either buffering records (reading from inputIterator) fails or - // thread reading records from buffer fails. - private final AtomicReference hasFailed = new AtomicReference(null); - // used for indicating that all the records from buffer are read successfully. - private final AtomicBoolean isDone = new AtomicBoolean(false); // next record to be read from buffer. private BufferedIteratorPayload nextRecord; - // schema used for fetching insertValue from HoodieRecord. - private final Schema schema; - public BufferedIterator(final Iterator iterator, final long bufferMemoryLimit, final Schema schema) { + public BufferedIterator(final Iterator iterator, final long bufferMemoryLimit, + final Schema schema) { this.inputIterator = iterator; this.bufferMemoryLimit = bufferMemoryLimit; this.schema = schema; @@ -92,23 +96,28 @@ public class BufferedIterator currentRateLimit) { rateLimiter.release(newRateLimit - currentRateLimit); @@ -120,12 +129,14 @@ public class BufferedIterator payload = new BufferedIteratorPayload<>(t, this.schema); buffer.put(Optional.of(payload)); @@ -198,12 +209,15 @@ public class BufferedIterator { + public T record; public Optional insertValue; // It caches the exception seen while fetching insert value. 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 8d305d214..c8a39b685 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 @@ -29,8 +29,8 @@ import org.apache.spark.api.java.function.Function2; /** * Map function that handles a sorted stream of HoodieRecords */ -public class BulkInsertMapFunction - implements Function2>, Iterator>> { +public class BulkInsertMapFunction implements + Function2>, Iterator>> { private String commitTime; private HoodieWriteConfig config; @@ -45,8 +45,7 @@ public class BulkInsertMapFunction @Override public Iterator> call(Integer partition, - Iterator> sortedRecordItr) - throws Exception { + Iterator> sortedRecordItr) throws Exception { return new LazyInsertIterable<>(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/LazyInsertIterable.java index 2406762a9..6e1ae02c4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java @@ -24,11 +24,6 @@ import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.io.HoodieCreateHandle; import com.uber.hoodie.io.HoodieIOHandle; import com.uber.hoodie.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.TaskContext; -import org.apache.spark.TaskContext$; - import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; @@ -38,6 +33,10 @@ import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; /** * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new @@ -68,35 +67,37 @@ public class LazyInsertIterable extends @Override protected List computeNext() { - // Need to set current spark thread's TaskContext into newly launched thread so that new thread can access + // Need to set current spark thread's TaskContext into newly launched thread so that new + // thread can access // TaskContext properties. final TaskContext sparkThreadTaskContext = TaskContext.get(); // Executor service used for launching writer thread. final ExecutorService writerService = Executors.newFixedThreadPool(1); try { - // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. - final BufferedIterator> bufferedIterator = - new BufferedIterator<>(inputItr, hoodieConfig.getWriteBufferLimitBytes(), - HoodieIOHandle.createHoodieWriteSchema(hoodieConfig)); - Future> writerResult = - writerService.submit( - () -> { - logger.info("starting hoodie writer thread"); - // Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext - // properties. - TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); - List statuses = new LinkedList<>(); - try { - statuses.addAll(handleWrite(bufferedIterator)); - logger.info("hoodie write is done; notifying reader thread"); - return statuses; - } catch (Exception e) { - logger.error("error writing hoodie records", e); - bufferedIterator.markAsFailed(e); - throw e; - } - }); - // Buffering records into internal buffer. This can throw exception either if reading records from spark fails or + // Used for buffering records which is controlled by + // HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. + final BufferedIterator> bufferedIterator = new BufferedIterator<>(inputItr, + hoodieConfig.getWriteBufferLimitBytes(), + HoodieIOHandle.createHoodieWriteSchema(hoodieConfig)); + Future> writerResult = writerService.submit(() -> { + logger.info("starting hoodie writer thread"); + // Passing parent thread's TaskContext to newly launched thread for it to access original + // TaskContext + // properties. + TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext); + List statuses = new LinkedList<>(); + try { + statuses.addAll(handleWrite(bufferedIterator)); + logger.info("hoodie write is done; notifying reader thread"); + return statuses; + } catch (Exception e) { + logger.error("error writing hoodie records", e); + bufferedIterator.markAsFailed(e); + throw e; + } + }); + // Buffering records into internal buffer. This can throw exception either if reading + // records from spark fails or // if writing buffered records into parquet file fails. bufferedIterator.startBuffering(); logger.info("waiting for hoodie write to finish"); @@ -110,28 +111,27 @@ public class LazyInsertIterable extends } } - private List handleWrite(final BufferedIterator> bufferedIterator) { + private List handleWrite( + final BufferedIterator> bufferedIterator) { List statuses = new ArrayList<>(); while (bufferedIterator.hasNext()) { - final BufferedIterator.BufferedIteratorPayload> payload = bufferedIterator.next(); + final BufferedIterator.BufferedIteratorPayload> payload = bufferedIterator + .next(); // clean up any partial failures if (!partitionsCleaned.contains(payload.record.getPartitionPath())) { // This insert task could fail multiple times, but Spark will faithfully retry with // the same data again. Thus, before we open any files under a given partition, we // first delete any files in the same partitionPath written by same Spark partition - HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, - commitTime, - payload.record.getPartitionPath(), - TaskContext.getPartitionId(), - hoodieTable); + HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, commitTime, + payload.record.getPartitionPath(), TaskContext.getPartitionId(), hoodieTable); partitionsCleaned.add(payload.record.getPartitionPath()); } // lazily initialize the handle, for the first time if (handle == null) { - handle = - new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, payload.record.getPartitionPath()); + handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, + payload.record.getPartitionPath()); } if (handle.canWrite(payload.record)) { @@ -141,9 +141,10 @@ public class LazyInsertIterable extends // handle is full. statuses.add(handle.close()); // Need to handle the rejected payload & open new handle - handle = - new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, payload.record.getPartitionPath()); - handle.write(payload.record, payload.insertValue, payload.exception); // we should be able to write 1 payload. + handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, + payload.record.getPartitionPath()); + handle.write(payload.record, payload.insertValue, + payload.exception); // we should be able to write 1 payload. } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyIterableIterator.java b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyIterableIterator.java index 2720e001e..4062a7dee 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyIterableIterator.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyIterableIterator.java @@ -23,9 +23,9 @@ import java.util.Iterator; * inputItr classes in order to simplify the implementation of lazy iterators for mapPartitions use * cases. Note [SPARK-3369], which gives the reasons for backwards compatibility with regard to the * iterable API despite Spark's single pass nature. - * + *

* Provide a way to obtain a inputItr of type O (output), out of an inputItr of type I (input) - * + *

* Things to remember: - Assumes Spark calls hasNext() to check for elements, before calling next() * to obtain them - Assumes hasNext() gets called atleast once. - Concrete Implementation is * responsible for calling inputIterator.next() and doing the processing in computeNext() 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 642cb7d9b..5733237b4 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 @@ -37,22 +37,30 @@ import org.apache.spark.api.java.JavaSparkContext; */ public abstract class HoodieIndex implements Serializable { - protected transient JavaSparkContext jsc = null; - - public enum IndexType { - HBASE, - INMEMORY, - BLOOM, - BUCKETED - } - 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()) { + case HBASE: + return new HBaseIndex<>(config, jsc); + case INMEMORY: + return new InMemoryHashIndex<>(config, jsc); + case BLOOM: + return new HoodieBloomIndex<>(config, jsc); + case BUCKETED: + return new BucketedIndex<>(config, jsc); + default: + throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); + } + } + /** * Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]] * If the optional FullFilePath value is not present, then the key is not found. If the @@ -71,7 +79,7 @@ public abstract class HoodieIndex implements Seri /** * 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, @@ -107,18 +115,7 @@ public abstract class HoodieIndex implements Seri public abstract boolean isImplicitWithStorage(); - public static HoodieIndex createIndex( - HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException { - switch (config.getIndexType()) { - case HBASE: - return new HBaseIndex<>(config, jsc); - case INMEMORY: - return new InMemoryHashIndex<>(config, jsc); - case BLOOM: - return new HoodieBloomIndex<>(config, jsc); - case BUCKETED: - return new BucketedIndex<>(config, jsc); - } - throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); + public enum IndexType { + HBASE, INMEMORY, BLOOM, BUCKETED } } 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 422d31983..6338e60bb 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 @@ -49,32 +49,11 @@ public class InMemoryHashIndex extends HoodieInde } @Override - public JavaPairRDD> fetchRecordLocation( - JavaRDD hoodieKeys, final HoodieTable table) { + public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, + final HoodieTable table) { throw new UnsupportedOperationException("InMemory index does not implement check exist yet"); } - /** - * Function that tags each HoodieRecord with an existing location, if known. - */ - class LocationTagFunction - implements Function2>, Iterator>> { - - @Override - public Iterator> call(Integer partitionNum, - Iterator> hoodieRecordIterator) { - List> taggedRecords = new ArrayList<>(); - while (hoodieRecordIterator.hasNext()) { - HoodieRecord rec = hoodieRecordIterator.next(); - if (recordLocationMap.containsKey(rec.getKey())) { - rec.setCurrentLocation(recordLocationMap.get(rec.getKey())); - } - taggedRecords.add(rec); - } - return taggedRecords.iterator(); - } - } - @Override public JavaRDD> tagLocation(JavaRDD> recordRDD, HoodieTable hoodieTable) { @@ -132,4 +111,25 @@ public class InMemoryHashIndex extends HoodieInde public boolean isImplicitWithStorage() { return false; } + + /** + * Function that tags each HoodieRecord with an existing location, if known. + */ + class LocationTagFunction implements + Function2>, Iterator>> { + + @Override + public Iterator> call(Integer partitionNum, + Iterator> hoodieRecordIterator) { + List> taggedRecords = new ArrayList<>(); + while (hoodieRecordIterator.hasNext()) { + HoodieRecord rec = hoodieRecordIterator.next(); + if (recordLocationMap.containsKey(rec.getKey())) { + rec.setCurrentLocation(recordLocationMap.get(rec.getKey())); + } + taggedRecords.add(rec); + } + return taggedRecords.iterator(); + } + } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/BloomIndexFileInfo.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/BloomIndexFileInfo.java index 0f0fb9908..6d398171b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/BloomIndexFileInfo.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/BloomIndexFileInfo.java @@ -64,8 +64,7 @@ public class BloomIndexFileInfo implements Serializable { * Does the given key fall within the range (inclusive) */ public boolean isKeyInRange(String recordKey) { - return minRecordKey.compareTo(recordKey) <= 0 && - maxRecordKey.compareTo(recordKey) >= 0; + return minRecordKey.compareTo(recordKey) <= 0 && maxRecordKey.compareTo(recordKey) >= 0; } @Override @@ -78,9 +77,8 @@ public class BloomIndexFileInfo implements Serializable { } BloomIndexFileInfo that = (BloomIndexFileInfo) o; - return Objects.equal(that.fileName, fileName) && - Objects.equal(that.minRecordKey, minRecordKey) && - Objects.equal(that.maxRecordKey, maxRecordKey); + return Objects.equal(that.fileName, fileName) && Objects.equal(that.minRecordKey, minRecordKey) + && Objects.equal(that.maxRecordKey, maxRecordKey); } 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 73a07fc4c..2b756c591 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 @@ -56,12 +56,11 @@ import scala.Tuple2; */ public class HoodieBloomIndex extends HoodieIndex { - private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class); - // we need to limit the join such that it stays within 1.5GB per Spark partition. (SPARK-1476) private static final int SPARK_MAXIMUM_BYTES_PER_PARTITION = 1500 * 1024 * 1024; // this is how much a triplet of (partitionPath, fileId, recordKey) costs. private static final int BYTES_PER_PARTITION_FILE_KEY_TRIPLET = 300; + private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class); private static int MAX_ITEMS_PER_SHUFFLE_PARTITION = SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET; @@ -108,32 +107,31 @@ public class HoodieBloomIndex extends HoodieIndex return taggedRecordRDD; } - public JavaPairRDD> fetchRecordLocation( - JavaRDD hoodieKeys, final HoodieTable table) { - JavaPairRDD partitionRecordKeyPairRDD = - hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey())); + public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, + final HoodieTable table) { + 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, + table); - JavaPairRDD rowKeyHoodieKeyPairRDD = - hoodieKeys.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key)); + JavaPairRDD rowKeyHoodieKeyPairRDD = hoodieKeys + .mapToPair(key -> new Tuple2<>(key.getRecordKey(), key)); - return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD) - .mapToPair(keyPathTuple -> { - Optional recordLocationPath; - if (keyPathTuple._2._2.isPresent()) { - String fileName = keyPathTuple._2._2.get(); - String partitionPath = keyPathTuple._2._1.getPartitionPath(); - recordLocationPath = Optional.of(new Path( - new Path(table.getMetaClient().getBasePath(), partitionPath), - fileName).toUri().getPath()); - } else { - recordLocationPath = Optional.absent(); - } - return new Tuple2<>(keyPathTuple._2._1, recordLocationPath); - }); + return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).mapToPair(keyPathTuple -> { + Optional recordLocationPath; + if (keyPathTuple._2._2.isPresent()) { + String fileName = keyPathTuple._2._2.get(); + String partitionPath = keyPathTuple._2._1.getPartitionPath(); + recordLocationPath = Optional + .of(new Path(new Path(table.getMetaClient().getBasePath(), partitionPath), fileName) + .toUri().getPath()); + } else { + recordLocationPath = Optional.absent(); + } + return new Tuple2<>(keyPathTuple._2._1, recordLocationPath); + }); } /** @@ -152,21 +150,21 @@ public class HoodieBloomIndex extends HoodieIndex final Map> partitionToFileInfo = fileInfoList.stream() .collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList()))); - // Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it. + // Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, + // that contains it. int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD); return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo, - partitionRecordKeyPairRDD, - parallelism); + partitionRecordKeyPairRDD, parallelism); } /** * The index lookup can be skewed in three dimensions : #files, #partitions, #records - * + *

* To be able to smoothly handle skews, we need to compute how to split each partitions into * subpartitions. We do it here, in a way that keeps the amount of each Spark join partition to < * 2GB. - * + *

* If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified * as a NON-zero number, then that is used explicitly. */ @@ -184,7 +182,8 @@ public class HoodieBloomIndex extends HoodieIndex // records for a partition. Map filesPerPartition = partitionToFileInfo.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> Long.valueOf(e.getValue().size()))); - long totalFiles = 0, totalRecords = 0; + long totalFiles = 0; + long totalRecords = 0; for (String partitionPath : recordsPerPartition.keySet()) { long numRecords = recordsPerPartition.get(partitionPath); long numFiles = @@ -210,22 +209,22 @@ public class HoodieBloomIndex extends HoodieIndex /** * Its crucial to pick the right parallelism. - * + *

* totalSubPartitions : this is deemed safe limit, to be nice with Spark. inputParallelism : * typically number of input file splits - * + *

* We pick the max such that, we are always safe, but go higher if say a there are a lot of input * files. (otherwise, we will fallback to number of partitions in input and end up with slow * performance) */ private int determineParallelism(int inputParallelism, int totalSubPartitions) { - // If bloom index parallelism is set, use it to to check against the input parallelism and take the max + // If bloom index parallelism is set, use it to to check against the input parallelism and + // take the max int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism()); int joinParallelism = Math.max(totalSubPartitions, indexParallelism); - logger.info("InputParallelism: ${" + inputParallelism + "}, " + - "IndexParallelism: ${" + config.getBloomIndexParallelism() + "}, " + - "TotalSubParts: ${" + totalSubPartitions + "}, " + - "Join Parallelism set to : " + joinParallelism); + logger.info("InputParallelism: ${" + inputParallelism + "}, " + "IndexParallelism: ${" + config + .getBloomIndexParallelism() + "}, " + "TotalSubParts: ${" + totalSubPartitions + "}, " + + "Join Parallelism set to : " + joinParallelism); return joinParallelism; } @@ -237,36 +236,31 @@ public class HoodieBloomIndex extends HoodieIndex final HoodieTable hoodieTable) { // 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().filterCompletedInstants().lastInstant(); + .parallelize(partitions, Math.max(partitions.size(), 1)).flatMapToPair(partitionPath -> { + java.util.Optional latestCommitTime = hoodieTable.getCommitsTimeline() + .filterCompletedInstants().lastInstant(); List> filteredFiles = new ArrayList<>(); if (latestCommitTime.isPresent()) { - filteredFiles = - hoodieTable.getROFileSystemView().getLatestDataFilesBeforeOrOn(partitionPath, - latestCommitTime.get().getTimestamp()) - .map(f -> new Tuple2<>(partitionPath, f)) - .collect(toList()); + filteredFiles = hoodieTable.getROFileSystemView() + .getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp()) + .map(f -> new Tuple2<>(partitionPath, f)).collect(toList()); } return filteredFiles.iterator(); }).collect(); if (config.getBloomIndexPruneByRanges()) { // also obtain file ranges, if range pruning is enabled - return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1)) - .mapToPair(ft -> { - try { - String[] minMaxKeys = ParquetUtils - .readMinMaxRecordKeys(hoodieTable.getHadoopConf(), - ft._2().getFileStatus().getPath()); - return new Tuple2<>(ft._1(), - new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1])); - } catch (MetadataNotFoundException me) { - logger.warn("Unable to find range metadata in file :" + ft._2()); - return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName())); - } - }).collect(); + return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1)).mapToPair(ft -> { + try { + String[] minMaxKeys = ParquetUtils + .readMinMaxRecordKeys(hoodieTable.getHadoopConf(), ft._2().getFileStatus().getPath()); + return new Tuple2<>(ft._1(), + new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1])); + } catch (MetadataNotFoundException me) { + logger.warn("Unable to find range metadata in file :" + ft._2()); + return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName())); + } + }).collect(); } else { return dataFilesList.stream() .map(ft -> new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName()))) @@ -320,62 +314,60 @@ public class HoodieBloomIndex extends HoodieIndex * (e.g: timestamp as prefix), the number of files to be compared gets cut down a lot from range * pruning. */ - // sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on recordKey + // sub-partition to ensure the records can be looked up against files & also prune + // file<=>record comparisons based on recordKey // ranges in the index info. @VisibleForTesting JavaPairRDD> explodeRecordRDDWithFileComparisons( final Map> partitionToFileIndexInfo, JavaPairRDD partitionRecordKeyPairRDD) { - return partitionRecordKeyPairRDD - .map(partitionRecordKeyPair -> { - String recordKey = partitionRecordKeyPair._2(); - String partitionPath = partitionRecordKeyPair._1(); + return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> { + String recordKey = partitionRecordKeyPair._2(); + String partitionPath = partitionRecordKeyPair._1(); - List indexInfos = partitionToFileIndexInfo.get(partitionPath); - List>> recordComparisons = new ArrayList<>(); - if (indexInfos - != null) { // could be null, if there are no files in a given partition yet. - // for each candidate file in partition, that needs to be compared. - for (BloomIndexFileInfo indexInfo : indexInfos) { - if (shouldCompareWithFile(indexInfo, recordKey)) { - recordComparisons.add( - new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey), - new Tuple2<>(indexInfo.getFileName(), - new HoodieKey(recordKey, partitionPath)))); - } - } + List indexInfos = partitionToFileIndexInfo.get(partitionPath); + List>> recordComparisons = new ArrayList<>(); + if (indexInfos != null) { // could be null, if there are no files in a given partition yet. + // for each candidate file in partition, that needs to be compared. + for (BloomIndexFileInfo indexInfo : indexInfos) { + if (shouldCompareWithFile(indexInfo, recordKey)) { + recordComparisons.add( + new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey), + new Tuple2<>(indexInfo.getFileName(), + new HoodieKey(recordKey, partitionPath)))); } - return recordComparisons; - }) - .flatMapToPair(t -> t.iterator()); + } + } + return recordComparisons; + }).flatMapToPair(t -> t.iterator()); } /** * Find out pair. All workload grouped by file-level. - * + *

* Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such * that each RDD partition is a file, then for each file, we do (1) load bloom filter, (2) load * rowKeys, (3) Tag rowKey - * + *

* Make sure the parallelism is atleast the groupby parallelism for tagging location */ @VisibleForTesting JavaPairRDD findMatchingFilesForRecordKeys(HoodieTable hoodieTable, final Map> partitionToFileIndexInfo, - JavaPairRDD partitionRecordKeyPairRDD, - int totalSubpartitions) { + JavaPairRDD partitionRecordKeyPairRDD, int totalSubpartitions) { int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), totalSubpartitions); - JavaPairRDD> fileSortedTripletRDD = explodeRecordRDDWithFileComparisons( - partitionToFileIndexInfo, partitionRecordKeyPairRDD) - // sort further based on filename, such that all checking for the file can happen within a single partition, on-the-fly - .sortByKey(true, joinParallelism); + JavaPairRDD> fileSortedTripletRDD = + explodeRecordRDDWithFileComparisons( + partitionToFileIndexInfo, partitionRecordKeyPairRDD) + // sort further based on filename, such that all checking for the file can happen within + // a single partition, on-the-fly + .sortByKey(true, joinParallelism); - return fileSortedTripletRDD - .mapPartitionsWithIndex( - new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true) + return fileSortedTripletRDD.mapPartitionsWithIndex( + new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true) .flatMap(indexLookupResults -> indexLookupResults.iterator()) .filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0) .flatMapToPair(lookupResult -> { @@ -391,25 +383,23 @@ public class HoodieBloomIndex extends HoodieIndex * Tag the back to the original HoodieRecord RDD. */ private JavaRDD> tagLocationBacktoRecords( - JavaPairRDD rowKeyFilenamePairRDD, - JavaRDD> recordRDD) { + JavaPairRDD rowKeyFilenamePairRDD, JavaRDD> recordRDD) { JavaPairRDD> rowKeyRecordPairRDD = recordRDD .mapToPair(record -> new Tuple2<>(record.getRecordKey(), record)); - // Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join. - return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map( - v1 -> { - HoodieRecord record = v1._1(); - if (v1._2().isPresent()) { - String filename = v1._2().get(); - if (filename != null && !filename.isEmpty()) { - record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename), - FSUtils.getFileId(filename))); - } - } - return record; + // Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), + // so we do left outer join. + return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(v1 -> { + HoodieRecord record = v1._1(); + if (v1._2().isPresent()) { + String filename = v1._2().get(); + if (filename != null && !filename.isEmpty()) { + record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename), + FSUtils.getFileId(filename))); } - ); + } + return record; + }); } @Override 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 cf5a12007..074ec56da 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 @@ -41,7 +41,8 @@ import scala.Tuple2; * actual files */ public class HoodieBloomIndexCheckFunction implements - Function2>>, Iterator>> { + Function2>>, + Iterator>> { private static Logger logger = LogManager.getLogger(HoodieBloomIndexCheckFunction.class); @@ -58,8 +59,7 @@ public class HoodieBloomIndexCheckFunction implements * Given a list of row keys and one file, return only row keys existing in that file. */ public static List checkCandidatesAgainstFile(Configuration configuration, - List candidateRecordKeys, - Path filePath) throws HoodieIndexException { + List candidateRecordKeys, Path filePath) throws HoodieIndexException { List foundRecordKeys = new ArrayList<>(); try { // Load all rowKeys from the file, to double-confirm @@ -86,6 +86,13 @@ public class HoodieBloomIndexCheckFunction implements return foundRecordKeys; } + @Override + public Iterator> call(Integer partition, + Iterator>> fileParitionRecordKeyTripletItr) + throws Exception { + return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr); + } + class LazyKeyCheckIterator extends LazyIterableIterator>, List> { @@ -143,7 +150,8 @@ public class HoodieBloomIndexCheckFunction implements // if continue on current file) if (fileName.equals(currentFile)) { - // check record key against bloom filter of current file & add to possible keys if needed + // check record key against bloom filter of current file & add to possible keys if + // needed if (bloomFilter.mightContain(recordKey)) { if (logger.isDebugEnabled()) { logger.debug("#1 Adding " + recordKey + " as candidate for file " + fileName); @@ -201,12 +209,4 @@ public class HoodieBloomIndexCheckFunction implements protected void end() { } } - - - @Override - public Iterator> call(Integer partition, - Iterator>> fileParitionRecordKeyTripletItr) - throws Exception { - return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr); - } } 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 a361a7b06..a8e392173 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 @@ -38,9 +38,9 @@ import scala.Tuple2; /** * An `stateless` index implementation that will using a deterministic mapping function to determine * the fileID for a given record. - * + *

* Pros: - Fast - * + *

* Cons : - Need to tune the number of buckets per partition path manually (FIXME: Need to autotune * this) - Could increase write amplification on copy-on-write storage since inserts always rewrite * files - Not global. 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 e2542925c..4928d1440 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 @@ -27,12 +27,16 @@ import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; -import com.uber.hoodie.config.HoodieIndexConfig; 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; +import java.util.LinkedList; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; @@ -51,23 +55,18 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function2; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; - /** * Hoodie Index implementation backed by HBase */ public class HBaseIndex extends HoodieIndex { - private final static byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s"); - private final static byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts"); - private final static byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name"); - private final static byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path"); + + private static final byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s"); + private static final byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts"); + private static final byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name"); + private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path"); private static Logger logger = LogManager.getLogger(HBaseIndex.class); - + private static Connection hbaseConnection = null; private final String tableName; public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) { @@ -77,14 +76,12 @@ public class HBaseIndex extends HoodieIndex { } @Override - public JavaPairRDD> fetchRecordLocation( - JavaRDD hoodieKeys, HoodieTable table) { + public JavaPairRDD> fetchRecordLocation(JavaRDD hoodieKeys, + HoodieTable table) { //TODO : Change/Remove filterExists in HoodieReadClient() and revisit throw new UnsupportedOperationException("HBase index does not implement check exist"); } - private static Connection hbaseConnection = null; - private Connection getHBaseConnection() { Configuration hbaseConfig = HBaseConfiguration.create(); String quorum = config.getHbaseZkQuorum(); @@ -100,15 +97,15 @@ public class HBaseIndex extends HoodieIndex { } /** - * Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is closed when - * JVM exits + * Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is + * closed when JVM exits */ private void addShutDownHook() { Runtime.getRuntime().addShutdownHook(new Thread() { public void run() { try { hbaseConnection.close(); - } catch(Exception e) { + } catch (Exception e) { // fail silently for any sort of exception } } @@ -126,101 +123,103 @@ public class HBaseIndex extends HoodieIndex { HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline(); // 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.containsInstant( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs)) || - HoodieTimeline.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), - commitTs, HoodieTimeline.GREATER)); + return !commitTimeline.empty() && (commitTimeline + .containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs)) + || HoodieTimeline + .compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs, + HoodieTimeline.GREATER)); } /** * Function that tags each HoodieRecord with an existing location, if known. */ private Function2>, Iterator>> - locationTagFunction(HoodieTable hoodieTable) { + locationTagFunction(HoodieTable hoodieTable) { return (Function2>, Iterator>>) (partitionNum, hoodieRecordIterator) -> { - Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize(); + Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize(); - // Grab the global HBase connection - synchronized (HBaseIndex.class) { - if (hbaseConnection == null || hbaseConnection.isClosed()) { - hbaseConnection = getHBaseConnection(); - } - } - List> taggedRecords = new ArrayList<>(); - HTable hTable = null; - try { - hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName)); - List statements = new ArrayList<>(); - List currentBatchOfRecords = new LinkedList<>(); - // Do the tagging. - while (hoodieRecordIterator.hasNext()) { - HoodieRecord rec = hoodieRecordIterator.next(); - statements.add(generateStatement(rec.getRecordKey())); - currentBatchOfRecords.add(rec); - // iterator till we reach batch size - if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) { - // get results for batch from Hbase - Result[] results = hTable.get(statements); - // clear statements to be GC'd - statements.clear(); - for (Result result : results) { - // first, attempt to grab location from HBase - HoodieRecord currentRecord = currentBatchOfRecords.remove(0); - if (result.getRow() != null) { - String keyFromResult = Bytes.toString(result.getRow()); - String commitTs = - Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)); - String fileId = - Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)); - String partitionPath = - Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN)); - - if (checkIfValidCommit(hoodieTable, commitTs)) { - currentRecord = new HoodieRecord(new HoodieKey(currentRecord.getRecordKey(), - partitionPath), currentRecord.getData()); - currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId)); - taggedRecords.add(currentRecord); - // the key from Result and the key being processed should be same - assert (currentRecord.getRecordKey().contentEquals(keyFromResult)); - } else { //if commit is invalid, treat this as a new taggedRecord - taggedRecords.add(currentRecord); - } - } else { - taggedRecords.add(currentRecord); - } + // Grab the global HBase connection + synchronized (HBaseIndex.class) { + if (hbaseConnection == null || hbaseConnection.isClosed()) { + hbaseConnection = getHBaseConnection(); } } - } - } catch (IOException e) { - throw new HoodieIndexException( - "Failed to Tag indexed locations because of exception with HBase Client", e); - } finally { - if (hTable != null) { + List> taggedRecords = new ArrayList<>(); + HTable hTable = null; try { - hTable.close(); - } catch (IOException e) { - // Ignore - } - } + hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName)); + List statements = new ArrayList<>(); + List currentBatchOfRecords = new LinkedList<>(); + // Do the tagging. + while (hoodieRecordIterator.hasNext()) { + HoodieRecord rec = hoodieRecordIterator.next(); + statements.add(generateStatement(rec.getRecordKey())); + currentBatchOfRecords.add(rec); + // iterator till we reach batch size + if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) { + // get results for batch from Hbase + Result[] results = hTable.get(statements); + // clear statements to be GC'd + statements.clear(); + for (Result result : results) { + // first, attempt to grab location from HBase + HoodieRecord currentRecord = currentBatchOfRecords.remove(0); + if (result.getRow() != null) { + String keyFromResult = Bytes.toString(result.getRow()); + String commitTs = Bytes + .toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)); + String fileId = Bytes + .toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)); + String partitionPath = Bytes + .toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN)); - } - return taggedRecords.iterator(); - }; + if (checkIfValidCommit(hoodieTable, commitTs)) { + currentRecord = new HoodieRecord( + new HoodieKey(currentRecord.getRecordKey(), partitionPath), + currentRecord.getData()); + currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId)); + taggedRecords.add(currentRecord); + // the key from Result and the key being processed should be same + assert (currentRecord.getRecordKey().contentEquals(keyFromResult)); + } else { //if commit is invalid, treat this as a new taggedRecord + taggedRecords.add(currentRecord); + } + } else { + taggedRecords.add(currentRecord); + } + } + } + } + } catch (IOException e) { + throw new HoodieIndexException( + "Failed to Tag indexed locations because of exception with HBase Client", e); + } finally { + if (hTable != null) { + try { + hTable.close(); + } catch (IOException e) { + // Ignore + } + } + + } + return taggedRecords.iterator(); + }; } @Override public JavaRDD> tagLocation(JavaRDD> recordRDD, - HoodieTable hoodieTable) { + HoodieTable hoodieTable) { return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable), true); } - private Function2, Iterator> updateLocationFunction() { - - return (Function2, Iterator>) (partition, statusIterator) -> { - + private Function2, Iterator> + updateLocationFunction() { + return (Function2, Iterator>) (partition, + statusIterator) -> { Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize(); List writeStatusList = new ArrayList<>(); @@ -292,16 +291,13 @@ public class HBaseIndex extends HoodieIndex { /** * Helper method to facilitate performing puts and deletes in Hbase - * @param hTable - * @param puts - * @param deletes - * @throws IOException */ - private void doPutsAndDeletes(HTable hTable, List puts, List deletes) throws IOException { - if(puts.size() > 0) { + private void doPutsAndDeletes(HTable hTable, List puts, List deletes) + throws IOException { + if (puts.size() > 0) { hTable.put(puts); } - if(deletes.size() > 0) { + if (deletes.size() > 0) { hTable.delete(deletes); } hTable.flushCommits(); @@ -311,7 +307,7 @@ public class HBaseIndex extends HoodieIndex { @Override public JavaRDD updateLocation(JavaRDD writeStatusRDD, - HoodieTable hoodieTable) { + HoodieTable hoodieTable) { return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true); } @@ -323,7 +319,6 @@ public class HBaseIndex extends HoodieIndex { /** * Only looks up by recordKey - * @return */ @Override public boolean isGlobal() { @@ -332,7 +327,6 @@ public class HBaseIndex extends HoodieIndex { /** * Mapping is available in HBase already. - * @return */ @Override public boolean canIndexLogFiles() { @@ -341,7 +335,6 @@ public class HBaseIndex extends HoodieIndex { /** * Index needs to be explicitly updated after storage write. - * @return */ @Override public boolean isImplicitWithStorage() { 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 4e62818b1..82d5a9d0b 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 @@ -37,14 +37,6 @@ import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieAppendException; import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.table.HoodieTable; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.TaskContext; -import org.apache.spark.util.SizeEstimator; - import java.io.IOException; import java.util.ArrayList; import java.util.Comparator; @@ -53,6 +45,13 @@ import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.TaskContext; +import org.apache.spark.util.SizeEstimator; /** * IO Operation to append data onto an existing file. @@ -61,14 +60,13 @@ public class HoodieAppendHandle extends HoodieIOH private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class); private static AtomicLong recordIndex = new AtomicLong(1); - - private TableFileSystemView.RealtimeView fileSystemView; private final WriteStatus writeStatus; private final String fileId; - private String partitionPath; - private Iterator> recordItr; List recordList = new ArrayList<>(); List keysToDelete = new ArrayList<>(); + private TableFileSystemView.RealtimeView fileSystemView; + private String partitionPath; + private Iterator> recordItr; private long recordsWritten = 0; private long recordsDeleted = 0; private long averageRecordSize = 0; @@ -76,11 +74,8 @@ public class HoodieAppendHandle extends HoodieIOH private Writer writer; private boolean doInit = true; - public HoodieAppendHandle(HoodieWriteConfig config, - String commitTime, - HoodieTable hoodieTable, - String fileId, - Iterator> recordItr) { + public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String fileId, Iterator> recordItr) { super(config, commitTime, hoodieTable); WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName()); writeStatus.setStat(new HoodieDeltaWriteStat()); @@ -93,42 +88,40 @@ public class HoodieAppendHandle extends HoodieIOH 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; + 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; - 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); - } - Path path = new Path(partitionPath, - FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)); - writeStatus.getStat().setPath(path.toString()); + 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); + } + Path path = new Path(partitionPath, + FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)); + writeStatus.getStat().setPath(path.toString()); } private Optional getIndexedRecord(HoodieRecord hoodieRecord) { @@ -150,7 +143,8 @@ public class HoodieAppendHandle extends HoodieIOH } writeStatus.markSuccess(hoodieRecord, recordMetadata); - // deflate record payload after recording success. This will help users access payload as a part of marking + // deflate record payload after recording success. This will help users access payload as a + // part of marking // record successful. hoodieRecord.deflate(); return avroRecord; @@ -165,7 +159,8 @@ public class HoodieAppendHandle extends HoodieIOH // 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; + 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()); @@ -173,16 +168,17 @@ public class HoodieAppendHandle extends HoodieIOH 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) { + 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 + 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; + averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2; doAppend(header); 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 85028ba29..c748dfe84 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 @@ -68,9 +68,8 @@ public class HoodieCleanHelper> { throws IOException { logger.info("Cleaning " + partitionPath + ", retaining latest " + config .getCleanerFileVersionsRetained() + " file versions. "); - List fileGroups = - fileSystemView.getAllFileGroups(partitionPath) - .collect(Collectors.toList()); + List fileGroups = fileSystemView.getAllFileGroups(partitionPath) + .collect(Collectors.toList()); List deletePaths = new ArrayList<>(); // Collect all the datafiles savepointed by all the savepoints List savepointedFiles = hoodieTable.getSavepoints().stream() @@ -94,11 +93,9 @@ public class HoodieCleanHelper> { FileSlice nextSlice = fileSliceIterator.next(); HoodieDataFile dataFile = nextSlice.getDataFile().get(); deletePaths.add(dataFile.getFileStatus().getPath().toString()); - if (hoodieTable.getMetaClient().getTableType() - == HoodieTableType.MERGE_ON_READ) { + if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { // If merge on read, then clean the log files for the commits as well - deletePaths.addAll(nextSlice.getLogFiles() - .map(file -> file.getPath().toString()) + deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString()) .collect(Collectors.toList())); } } @@ -121,8 +118,8 @@ public class HoodieCleanHelper> { private List getFilesToCleanKeepingLatestCommits(String partitionPath) throws IOException { int commitsRetained = config.getCleanerCommitsRetained(); - logger.info( - "Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. "); + logger + .info("Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. "); List deletePaths = new ArrayList<>(); // Collect all the datafiles savepointed by all the savepoints @@ -132,15 +129,14 @@ public class HoodieCleanHelper> { // determine if we have enough commits, to start cleaning. if (commitTimeline.countInstants() > commitsRetained) { HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get(); - List fileGroups = - fileSystemView.getAllFileGroups(partitionPath) - .collect(Collectors.toList()); + List fileGroups = fileSystemView.getAllFileGroups(partitionPath) + .collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { List fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList()); HoodieDataFile dataFile = fileSliceList.get(0).getDataFile().get(); String lastVersion = dataFile.getCommitTime(); - String lastVersionBeforeEarliestCommitToRetain = - getLatestVersionBeforeCommit(fileSliceList, earliestCommitToRetain); + String lastVersionBeforeEarliestCommitToRetain = getLatestVersionBeforeCommit(fileSliceList, + earliestCommitToRetain); // Ensure there are more than 1 version of the file (we only clean old files from updates) // i.e always spare the last commit. @@ -151,28 +147,26 @@ public class HoodieCleanHelper> { // do not clean up a savepoint data file continue; } - // Dont delete the latest commit and also the last commit before the earliest commit we are retaining - // The window of commit retain == max query run time. So a query could be running which still + // Dont delete the latest commit and also the last commit before the earliest commit we + // are retaining + // The window of commit retain == max query run time. So a query could be running which + // still // uses this file. - if (fileCommitTime.equals(lastVersion) || ( - lastVersionBeforeEarliestCommitToRetain != null && fileCommitTime - .equals(lastVersionBeforeEarliestCommitToRetain))) { + if (fileCommitTime.equals(lastVersion) || (lastVersionBeforeEarliestCommitToRetain != null + && fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) { // move on to the next file continue; } // Always keep the last commit - if (HoodieTimeline.compareTimestamps( - earliestCommitToRetain.getTimestamp(), - fileCommitTime, - HoodieTimeline.GREATER)) { + if (HoodieTimeline + .compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime, + HoodieTimeline.GREATER)) { // this is a commit, that should be cleaned. deletePaths.add(aFile.getFileStatus().getPath().toString()); - if (hoodieTable.getMetaClient().getTableType() - == HoodieTableType.MERGE_ON_READ) { + if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { // If merge on read, then clean the log files for the commits as well - deletePaths.addAll(aSlice.getLogFiles() - .map(file -> file.getPath().toString()) + deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString()) .collect(Collectors.toList())); } } @@ -190,9 +184,10 @@ public class HoodieCleanHelper> { HoodieInstant commitTime) { for (FileSlice file : fileSliceList) { String fileCommitTime = file.getDataFile().get().getCommitTime(); - if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime, - HoodieTimeline.GREATER)) { - // fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want + if (HoodieTimeline + .compareTimestamps(commitTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) { + // fileList is sorted on the reverse, so the first commit we find <= commitTime is the + // one we want return fileCommitTime; } } @@ -213,8 +208,7 @@ public class HoodieCleanHelper> { } else { throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name()); } - logger.info( - deletePaths.size() + " patterns used to delete in partition path:" + partitionPath); + logger.info(deletePaths.size() + " patterns used to delete in partition path:" + partitionPath); return deletePaths; } @@ -227,8 +221,8 @@ public class HoodieCleanHelper> { int commitsRetained = config.getCleanerCommitsRetained(); if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS && commitTimeline.countInstants() > commitsRetained) { - earliestCommitToRetain = - commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained); + earliestCommitToRetain = commitTimeline + .nthInstant(commitTimeline.countInstants() - commitsRetained); } return earliestCommitToRetain; } 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 d0454e3d2..10d9f94cc 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 @@ -40,13 +40,6 @@ import com.uber.hoodie.exception.HoodieCommitException; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.table.HoodieTable; -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileStream; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -54,6 +47,11 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** * Archiver to bound the growth of .commit files @@ -76,11 +74,9 @@ public class HoodieCommitArchiveLog { private HoodieLogFormat.Writer openWriter() { try { if (this.writer == null) { - return HoodieLogFormat.newWriterBuilder() - .onParentPath(archiveFilePath.getParent()) + return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent()) .withFileId(archiveFilePath.getName()) - .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION) - .withFs(metaClient.getFs()) + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFs(metaClient.getFs()) .overBaseCommit("").build(); } else { return this.writer; @@ -136,21 +132,19 @@ public class HoodieCommitArchiveLog { .getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)) .filterCompletedInstants(); Stream instants = cleanAndRollbackTimeline.getInstants() - .collect(Collectors.groupingBy(s -> s.getAction())) - .entrySet() - .stream() - .map(i -> { + .collect(Collectors.groupingBy(s -> s.getAction())).entrySet().stream().map(i -> { if (i.getValue().size() > maxCommitsToKeep) { return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep); } else { return new ArrayList(); } - }) - .flatMap(i -> i.stream()); + }).flatMap(i -> i.stream()); - //TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats + //TODO (na) : Add a way to return actions associated with a timeline and then merge/unify + // with logic above to avoid Stream.concats HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); - // We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present. + // We cannot have any holes in the commit timeline. We cannot archive any commits which are + // made after the first savepoint present. Optional firstSavepoint = table.getCompletedSavepointTimeline().firstInstant(); if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) { // Actually do the commits @@ -169,16 +163,14 @@ public class HoodieCommitArchiveLog { log.info("Deleting instants " + archivedInstants); boolean success = true; for (HoodieInstant archivedInstant : archivedInstants) { - Path commitFile = - new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); + Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); try { if (metaClient.getFs().exists(commitFile)) { success &= metaClient.getFs().delete(commitFile, false); log.info("Archived and deleted instant file " + commitFile); } } catch (IOException e) { - throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, - e); + throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e); } } return success; @@ -186,8 +178,8 @@ public class HoodieCommitArchiveLog { public void archive(List instants) throws HoodieCommitException { try { - HoodieTimeline commitTimeline = - metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants(); + HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline() + .filterCompletedInstants(); Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); log.info("Wrapper schema " + wrapperSchema.toString()); List records = new ArrayList<>(); @@ -247,6 +239,8 @@ public class HoodieCommitArchiveLog { archivedMetaWrapper.setActionType(ActionType.commit.name()); break; } + default: + throw new UnsupportedOperationException("Action not fully supported yet"); } return archivedMetaWrapper; } @@ -256,9 +250,8 @@ public class HoodieCommitArchiveLog { ObjectMapper mapper = new ObjectMapper(); //Need this to ignore other public get() methods 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); + com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData = mapper + .convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class); return avroMetaData; } } 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 9c7fac34c..f16b2d211 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 @@ -49,8 +49,8 @@ public class HoodieCreateHandle extends HoodieIOH private long recordsWritten = 0; private long recordsDeleted = 0; - public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, - HoodieTable hoodieTable, String partitionPath) { + public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String partitionPath) { super(config, commitTime, hoodieTable); this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName()); status.setFileId(UUID.randomUUID().toString()); @@ -64,14 +64,11 @@ public class HoodieCreateHandle extends HoodieIOH } try { - HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, - commitTime, - new Path(config.getBasePath()), - new Path(config.getBasePath(), partitionPath)); + HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime, + new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath)); partitionMetadata.trySave(TaskContext.getPartitionId()); - this.storageWriter = - HoodieStorageWriterFactory - .getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema); + this.storageWriter = HoodieStorageWriterFactory + .getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema); } catch (IOException e) { throw new HoodieInsertException( "Failed to initialize HoodieStorageWriter for path " + getStorageWriterPath(), e); @@ -81,13 +78,12 @@ public class HoodieCreateHandle extends HoodieIOH /** * 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 storageWriter.canWrite() && record.getPartitionPath() - .equals(status.getPartitionPath()); + return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath()); } /** @@ -111,7 +107,8 @@ public class HoodieCreateHandle extends HoodieIOH recordsDeleted++; } status.markSuccess(record, recordMetadata); - // deflate record payload after recording success. This will help users access payload as a part of marking + // deflate record payload after recording success. This will help users access payload as a + // part of marking // record successful. record.deflate(); } catch (Throwable t) { @@ -126,9 +123,8 @@ public class HoodieCreateHandle extends HoodieIOH * Performs actions to durably, persist the current changes and returns a WriteStatus object */ public WriteStatus close() { - logger.info( - "Closing the file " + status.getFileId() + " as we are done with all the records " - + recordsWritten); + logger.info("Closing the file " + status.getFileId() + " as we are done with all the records " + + recordsWritten); try { storageWriter.close(); @@ -144,8 +140,7 @@ public class HoodieCreateHandle extends HoodieIOH return status; } catch (IOException e) { - throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, - e); + throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e); } } 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 f9795ceae..a0032de2b 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 @@ -39,11 +39,10 @@ public abstract class HoodieIOHandle { protected final HoodieWriteConfig config; protected final FileSystem fs; protected final HoodieTable hoodieTable; - protected HoodieTimeline hoodieTimeline; protected final Schema schema; + protected HoodieTimeline hoodieTimeline; - public HoodieIOHandle(HoodieWriteConfig config, String commitTime, - HoodieTable hoodieTable) { + public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable) { this.commitTime = commitTime; this.config = config; this.fs = hoodieTable.getMetaClient().getFs(); @@ -52,6 +51,32 @@ public abstract class HoodieIOHandle { this.schema = createHoodieWriteSchema(config); } + /** + * Deletes any new tmp files written during the current commit, into the partition + */ + public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config, String commitTime, + String partitionPath, int taskPartitionId, HoodieTable hoodieTable) { + FileSystem fs = hoodieTable.getMetaClient().getFs(); + try { + FileStatus[] prevFailedFiles = fs.globStatus(new Path(String + .format("%s/%s/%s", config.getBasePath(), partitionPath, + FSUtils.maskWithoutFileId(commitTime, taskPartitionId)))); + if (prevFailedFiles != null) { + logger.info( + "Deleting " + prevFailedFiles.length + " files generated by previous failed attempts."); + for (FileStatus status : prevFailedFiles) { + fs.delete(status.getPath(), false); + } + } + } catch (IOException e) { + throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime, e); + } + } + + public static Schema createHoodieWriteSchema(HoodieWriteConfig config) { + return HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + } + public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) { Path path = new Path(config.getBasePath(), partitionPath); try { @@ -72,37 +97,7 @@ public abstract class HoodieIOHandle { taskAttemptId)); } - /** - * Deletes any new tmp files written during the current commit, into the partition - */ - public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config, - String commitTime, - String partitionPath, - int taskPartitionId, - HoodieTable hoodieTable) { - FileSystem fs = hoodieTable.getMetaClient().getFs(); - try { - FileStatus[] prevFailedFiles = fs.globStatus(new Path(String - .format("%s/%s/%s", config.getBasePath(), partitionPath, - FSUtils.maskWithoutFileId(commitTime, taskPartitionId)))); - if (prevFailedFiles != null) { - logger.info("Deleting " + prevFailedFiles.length - + " files generated by previous failed attempts."); - for (FileStatus status : prevFailedFiles) { - fs.delete(status.getPath(), false); - } - } - } catch (IOException e) { - throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime, - e); - } - } - public Schema getSchema() { return schema; } - - public static Schema createHoodieWriteSchema(HoodieWriteConfig config) { - return HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); - } } 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 c5ee6da8b..dc2930aae 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 @@ -26,14 +26,18 @@ import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; +import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieUpsertException; import com.uber.hoodie.io.storage.HoodieStorageWriter; import com.uber.hoodie.io.storage.HoodieStorageWriterFactory; import com.uber.hoodie.table.HoodieTable; +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; +import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; @@ -41,11 +45,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.TaskContext; -import java.io.IOException; -import java.util.Iterator; -import java.util.Map; -import java.util.Optional; - @SuppressWarnings("Duplicates") public class HoodieMergeHandle extends HoodieIOHandle { @@ -62,59 +61,46 @@ public class HoodieMergeHandle extends HoodieIOHa private long recordsDeleted = 0; private long updatedRecordsWritten = 0; - public HoodieMergeHandle(HoodieWriteConfig config, - String commitTime, - HoodieTable hoodieTable, - Iterator> recordItr, - String fileId) { + public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + Iterator> recordItr, String fileId) { super(config, commitTime, hoodieTable); this.fileSystemView = hoodieTable.getROFileSystemView(); init(fileId, init(fileId, recordItr)); } - public HoodieMergeHandle(HoodieWriteConfig config, - String commitTime, - HoodieTable hoodieTable, - Map> keyToNewRecords, - String fileId) { + public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + Map> keyToNewRecords, String fileId) { super(config, commitTime, hoodieTable); this.fileSystemView = hoodieTable.getROFileSystemView(); this.keyToNewRecords = keyToNewRecords; - init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get()).getPartitionPath()); + init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get()) + .getPartitionPath()); } /** * Extract old file path, initialize StorageWriter and WriteStatus - * @param fileId - * @param partitionPath */ private void init(String fileId, String partitionPath) { WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName()); writeStatus.setStat(new HoodieWriteStat()); this.writeStatus = writeStatus; try { - String latestValidFilePath = fileSystemView - .getLatestDataFiles(partitionPath) - .filter(dataFile -> dataFile.getFileId().equals(fileId)) - .findFirst() - .get().getFileName(); + String latestValidFilePath = fileSystemView.getLatestDataFiles(partitionPath) + .filter(dataFile -> dataFile.getFileId().equals(fileId)).findFirst().get().getFileName(); writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath)); - HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, - commitTime, - new Path(config.getBasePath()), - new Path(config.getBasePath(), partitionPath)); + HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime, + new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath)); partitionMetadata.trySave(TaskContext.getPartitionId()); oldFilePath = new Path( - config.getBasePath() + "/" + partitionPath + "/" - + latestValidFilePath); + config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath); String relativePath = new Path(partitionPath + "/" + FSUtils .makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString(); newFilePath = new Path(config.getBasePath(), relativePath); if (config.shouldUseTempFolderForCopyOnWriteForMerge()) { - this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(), - fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); + this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(), fileId, + TaskContext.get().stageId(), TaskContext.get().taskAttemptId()); } // handle cases of partial failures, for update task @@ -122,8 +108,9 @@ public class HoodieMergeHandle extends HoodieIOHa fs.delete(newFilePath, false); } - logger.info(String.format("Merging new data into oldPath %s, as newPath %s", - oldFilePath.toString(), getStorageWriterPath().toString())); + logger.info(String + .format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(), + getStorageWriterPath().toString())); // file name is same for all records, in this bunch writeStatus.setFileId(fileId); writeStatus.setPartitionPath(partitionPath); @@ -143,9 +130,6 @@ public class HoodieMergeHandle extends HoodieIOHa /** * Load the new incoming records in a map and return partitionPath - * @param fileId - * @param newRecordsItr - * @return */ private String init(String fileId, Iterator> newRecordsItr) { try { @@ -153,7 +137,7 @@ public class HoodieMergeHandle extends HoodieIOHa logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge()); this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(), Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, config.getPayloadClass())); - } catch(IOException io) { + } catch (IOException io) { throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io); } String partitionPath = null; @@ -164,14 +148,14 @@ public class HoodieMergeHandle extends HoodieIOHa // update the new location of the record, so we know where to find it next record.setNewLocation(new HoodieRecordLocation(commitTime, fileId)); } - logger.debug("Number of entries in MemoryBasedMap => " + - ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries() - + "Total size in bytes of MemoryBasedMap => " + - ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize() - + "Number of entries in DiskBasedMap => " + - ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries() - + "Size of file spilled to disk => " + - ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); + logger.debug("Number of entries in MemoryBasedMap => " + + ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries() + + "Total size in bytes of MemoryBasedMap => " + + ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize() + + "Number of entries in DiskBasedMap => " + + ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries() + + "Size of file spilled to disk => " + + ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); return partitionPath; } @@ -189,7 +173,8 @@ public class HoodieMergeHandle extends HoodieIOHa } writeStatus.markSuccess(hoodieRecord, recordMetadata); - // deflate record payload after recording success. This will help users access payload as a part of marking + // deflate record payload after recording success. This will help users access payload as a + // part of marking // record successful. hoodieRecord.deflate(); return true; @@ -201,8 +186,7 @@ public class HoodieMergeHandle extends HoodieIOHa } /** - * Go through an old record. Here if we detect a newer version shows up, we write the new one to - * the file. + * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file. */ public void write(GenericRecord oldRecord) { String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); @@ -213,12 +197,12 @@ public class HoodieMergeHandle extends HoodieIOHa Optional combinedAvroRecord = hoodieRecord.getData() .combineAndGetUpdateValue(oldRecord, schema); if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) { - /* ONLY WHEN - * 1) we have an update for this key AND - * 2) We are able to successfully write the the combined new value - * - * We no longer need to copy the old record over. - */ + /* ONLY WHEN + * 1) we have an update for this key AND + * 2) We are able to successfully write the the combined new value + * + * We no longer need to copy the old record over. + */ copyOldRecord = false; } keyToNewRecords.remove(key); @@ -236,10 +220,9 @@ public class HoodieMergeHandle extends HoodieIOHa try { storageWriter.writeAvro(key, oldRecord); } catch (ClassCastException e) { - logger.error( - "Schema mismatch when rewriting old record " + oldRecord + " from file " - + getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema - .toString(true)); + logger.error("Schema mismatch when rewriting old record " + oldRecord + " from file " + + getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema + .toString(true)); throw new HoodieUpsertException(errMsg, e); } catch (IOException e) { logger.error("Failed to merge old record into new file for key " + key + " from old file " 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 7304c11d8..8180f7dc1 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 @@ -53,8 +53,8 @@ public class CompactionOperation implements Serializable { this.partitionPath = partitionPath; this.dataFileCommitTime = dataFile.getCommitTime(); this.dataFileSize = dataFile.getFileSize(); - this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()).collect( - Collectors.toList()); + this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()) + .collect(Collectors.toList()); this.metrics = writeConfig.getCompactionStrategy() .captureMetrics(dataFile, partitionPath, logFiles); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java index 9011a82b2..7475176c4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieCompactor.java @@ -17,17 +17,15 @@ package com.uber.hoodie.io.compact; import com.uber.hoodie.WriteStatus; -import com.uber.hoodie.common.model.HoodieCommitMetadata; 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.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieTable; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - import java.io.Serializable; import java.util.Date; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; /** * A HoodieCompactor runs compaction on a hoodie table @@ -38,7 +36,7 @@ public interface HoodieCompactor extends Serializable { * Compact the delta files with the data files */ JavaRDD compact(JavaSparkContext jsc, final HoodieWriteConfig config, - HoodieTable hoodieTable, String compactionCommitTime) throws Exception; + HoodieTable hoodieTable, String compactionCommitTime) throws Exception; // Helper methods 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 094ee0200..8d4f9b7a3 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 @@ -17,6 +17,7 @@ package com.uber.hoodie.io.compact; import static java.util.stream.Collectors.toList; + import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -70,9 +71,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { } private JavaRDD executeCompaction(JavaSparkContext jsc, - List operations, - HoodieTable hoodieTable, - HoodieWriteConfig config, String compactionCommitTime) throws IOException { + List operations, HoodieTable hoodieTable, HoodieWriteConfig config, + String compactionCommitTime) throws IOException { log.info("After filtering, Compacting " + operations + " files"); return jsc.parallelize(operations, operations.size()) @@ -80,18 +80,19 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { .flatMap(writeStatusesItr -> writeStatusesItr.iterator()); } - private List compact(HoodieTable hoodieTable, - HoodieWriteConfig config, CompactionOperation operation, String commitTime) - throws IOException { + private List compact(HoodieTable hoodieTable, HoodieWriteConfig config, + CompactionOperation operation, String commitTime) throws IOException { FileSystem fs = hoodieTable.getMetaClient().getFs(); - Schema readerSchema = - HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + Schema readerSchema = HoodieAvroUtils + .addMetadataFields(new Schema.Parser().parse(config.getSchema())); log.info("Compacting base " + operation.getDataFilePath() + " with delta files " + operation .getDeltaFilePaths() + " for commit " + commitTime); // TODO - FIX THIS - // Reads the entire avro file. Always only specific blocks should be read from the avro file (failure recover). - // Load all the delta commits since the last compaction commit and get all the blocks to be loaded and load it using CompositeAvroLogReader + // Reads the entire avro file. Always only specific blocks should be read from the avro file + // (failure recover). + // 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() @@ -114,52 +115,47 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { Iterator> result = table .handleUpdate(commitTime, operation.getFileId(), scanner.getRecords()); Iterable> resultIterable = () -> result; - return StreamSupport.stream(resultIterable.spliterator(), false) - .flatMap(Collection::stream) + return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream) .map(s -> { s.getStat().setTotalRecordsToBeUpdate(scanner.getTotalRecordsToUpdate()); s.getStat().setTotalLogFiles(scanner.getTotalLogFiles()); s.getStat().setTotalLogRecords(scanner.getTotalLogRecords()); s.getStat().setPartitionPath(operation.getPartitionPath()); return s; - }) - .collect(toList()); + }).collect(toList()); } private List getCompactionWorkload(JavaSparkContext jsc, - HoodieTable hoodieTable, - HoodieWriteConfig config, String compactionCommitTime) + HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime) throws IOException { - Preconditions.checkArgument( - hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, - "HoodieRealtimeTableCompactor can only compact table of type " - + HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient() - .getTableType().name()); + Preconditions + .checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, + "HoodieRealtimeTableCompactor can only compact table of type " + + HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient() + .getTableType().name()); //TODO : check if maxMemory is not greater than JVM or spark.executor memory // TODO - rollback any compactions in flight HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); - List partitionPaths = - FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), + List partitionPaths = FSUtils + .getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning()); TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView(); log.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); - List operations = - jsc.parallelize(partitionPaths, partitionPaths.size()) - .flatMap((FlatMapFunction) partitionPath -> fileSystemView - .getLatestFileSlices(partitionPath) - .map(s -> new CompactionOperation(s.getDataFile().get(), - partitionPath, + List operations = jsc.parallelize(partitionPaths, partitionPaths.size()) + .flatMap((FlatMapFunction) partitionPath -> fileSystemView + .getLatestFileSlices(partitionPath).map( + s -> new CompactionOperation(s.getDataFile().get(), partitionPath, s.getLogFiles().sorted(HoodieLogFile.getLogVersionComparator().reversed()) .collect(Collectors.toList()), config)) - .filter(c -> !c.getDeltaFilePaths().isEmpty()) - .collect(toList()).iterator()).collect(); + .filter(c -> !c.getDeltaFilePaths().isEmpty()).collect(toList()).iterator()).collect(); log.info("Total of " + operations.size() + " compactions are retrieved"); - // Filter the compactions with the passed in filter. This lets us choose most effective compactions only + // Filter the compactions with the passed in filter. This lets us choose most effective + // compactions only operations = config.getCompactionStrategy().orderAndFilter(config, operations); if (operations.isEmpty()) { log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java index 676b3e10e..9393a0479 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/BoundedIOCompactionStrategy.java @@ -44,9 +44,9 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy { List logFiles) { Map metrics = Maps.newHashMap(); // 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); + 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); // Total write will be similar to the size of the base file @@ -64,7 +64,8 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy { @Override public List orderAndFilter(HoodieWriteConfig writeConfig, List operations) { - // Iterate through the operations in order and accept operations as long as we are within the IO limit + // Iterate through the operations in order and accept operations as long as we are within the + // IO limit // Preserves the original ordering of compactions List finalOperations = Lists.newArrayList(); long targetIORemaining = writeConfig.getTargetIOPerCompactionInMB(); 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 8486774d7..c5a666cbd 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 @@ -28,7 +28,7 @@ import java.util.Map; * Strategy for compaction. Pluggable implementation of define how compaction should be done. The * implementations of this interface can capture the relevant metrics to order and filter the final * list of compaction operation to run in a single compaction. - * + *

* Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be * passed in every time * @@ -41,9 +41,9 @@ public interface CompactionStrategy extends Serializable { * Callback hook when a CompactionOperation is created. Individual strategies can capture the * metrics they need to decide on the priority. * - * @param dataFile - Base file to compact + * @param dataFile - Base file to compact * @param partitionPath - Partition path - * @param logFiles - List of log files to compact with the base file + * @param logFiles - List of log files to compact with the base file * @return Map[String, Object] - metrics captured */ Map captureMetrics(HoodieDataFile dataFile, String partitionPath, @@ -54,7 +54,7 @@ public interface CompactionStrategy extends Serializable { * order and filter out compactions * * @param writeConfig - HoodieWriteConfig - config for this compaction is passed in - * @param operations - list of compactions collected + * @param operations - list of compactions collected * @return list of compactions to perform in this run */ List orderAndFilter(HoodieWriteConfig writeConfig, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java index 5aa967869..3e5045952 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/DayBasedCompactionStrategy.java @@ -20,7 +20,6 @@ package com.uber.hoodie.io.compact.strategy; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.io.compact.CompactionOperation; - import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Comparator; @@ -30,20 +29,21 @@ import java.util.Locale; import java.util.stream.Collectors; /** - * This strategy orders compactions in reverse order of creation of Hive Partitions. - * It helps to compact data in latest partitions first and then older capped at the Total_IO allowed. + * This strategy orders compactions in reverse order of creation of Hive Partitions. It helps to + * compact data in latest partitions first and then older capped at the Total_IO allowed. */ public class DayBasedCompactionStrategy extends BoundedIOCompactionStrategy { // For now, use SimpleDateFormat as default partition format private static String datePartitionFormat = "yyyy/MM/dd"; // Sorts compaction in LastInFirstCompacted order - private static Comparator comparator = (CompactionOperation leftC, CompactionOperation rightC) -> { + private static Comparator comparator = (CompactionOperation leftC, + CompactionOperation rightC) -> { try { Date left = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH) - .parse(leftC.getPartitionPath()); + .parse(leftC.getPartitionPath()); Date right = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH) - .parse(rightC.getPartitionPath()); + .parse(rightC.getPartitionPath()); return left.after(right) ? -1 : right.after(left) ? 1 : 0; } catch (ParseException e) { throw new HoodieException("Invalid Partition Date Format", e); @@ -55,8 +55,10 @@ public class DayBasedCompactionStrategy extends BoundedIOCompactionStrategy { } @Override - public List orderAndFilter(HoodieWriteConfig writeConfig, List operations) { + public List orderAndFilter(HoodieWriteConfig writeConfig, + List operations) { // Iterate through the operations and accept operations as long as we are within the IO limit - return super.orderAndFilter(writeConfig, operations.stream().sorted(comparator).collect(Collectors.toList())); + return super.orderAndFilter(writeConfig, + operations.stream().sorted(comparator).collect(Collectors.toList())); } } \ No newline at end of file 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 2fabc6596..a7f729e36 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 @@ -44,9 +44,9 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat Map metrics = super.captureMetrics(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).orElse(0L); + Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize) + .filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2) + .orElse(0L); // save the metrics needed during the order metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize); return metrics; 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 9978c0747..edd2c0620 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 @@ -36,8 +36,8 @@ 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 HoodieParquetWriter - extends ParquetWriter implements HoodieStorageWriter { +public class HoodieParquetWriter extends + ParquetWriter implements HoodieStorageWriter { private static AtomicLong recordIndex = new AtomicLong(1); @@ -49,6 +49,29 @@ public class HoodieParquetWriter HoodieStorageWriter getStorageWriter( - String commitTime, Path path, HoodieTable hoodieTable, HoodieWriteConfig config, - Schema schema) - throws IOException { + 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( + 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(new AvroSchemaConverter().convert(schema), schema, filter); + BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(), + config.getBloomFilterFPP()); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( + new AvroSchemaConverter().convert(schema), schema, filter); HoodieParquetConfig parquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java index a9d2a2298..390e73abe 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java @@ -59,8 +59,8 @@ import org.apache.hadoop.util.Progressable; */ public class HoodieWrapperFileSystem extends FileSystem { - private static final Set SUPPORT_SCHEMES; public static final String HOODIE_SCHEME_PREFIX = "hoodie-"; + private static final Set SUPPORT_SCHEMES; static { SUPPORT_SCHEMES = new HashSet<>(); @@ -69,18 +69,50 @@ public class HoodieWrapperFileSystem extends FileSystem { SUPPORT_SCHEMES.add("s3"); SUPPORT_SCHEMES.add("s3a"); - // Hoodie currently relies on underlying object store being fully // consistent so only regional buckets should be used. SUPPORT_SCHEMES.add("gs"); SUPPORT_SCHEMES.add("viewfs"); } - private ConcurrentMap openStreams = - new ConcurrentHashMap<>(); + private ConcurrentMap openStreams = new + ConcurrentHashMap<>(); private FileSystem fileSystem; private URI uri; + public static Path convertToHoodiePath(Path file, Configuration conf) { + try { + String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); + return convertPathWithScheme(file, getHoodieScheme(scheme)); + } catch (HoodieIOException e) { + throw e; + } + } + + private static Path convertPathWithScheme(Path oldPath, String newScheme) { + URI oldURI = oldPath.toUri(); + URI newURI; + try { + newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(), + oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment()); + return new Path(newURI); + } catch (URISyntaxException e) { + // TODO - Better Exception handling + throw new RuntimeException(e); + } + } + + public static String getHoodieScheme(String scheme) { + String newScheme; + if (SUPPORT_SCHEMES.contains(scheme)) { + newScheme = HOODIE_SCHEME_PREFIX + scheme; + } else { + throw new IllegalArgumentException( + "BlockAlignedAvroParquetWriter does not support scheme " + scheme); + } + return newScheme; + } + @Override public void initialize(URI uri, Configuration conf) throws IOException { // Get the default filesystem to decorate @@ -90,7 +122,8 @@ public class HoodieWrapperFileSystem extends FileSystem { path = new Path(path.toString().replace(HOODIE_SCHEME_PREFIX, "")); } this.fileSystem = FSUtils.getFs(path.toString(), conf); - // Do not need to explicitly initialize the default filesystem, its done already in the above FileSystem.get + // Do not need to explicitly initialize the default filesystem, its done already in the above + // FileSystem.get // fileSystem.initialize(FileSystem.getDefaultUri(conf), conf); // fileSystem.setConf(conf); this.uri = uri; @@ -108,8 +141,7 @@ public class HoodieWrapperFileSystem extends FileSystem { @Override public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, - int bufferSize, short replication, long blockSize, Progressable progress) - throws IOException { + int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { final Path translatedPath = convertToDefaultPath(f); return wrapOutputStream(f, fileSystem .create(translatedPath, permission, overwrite, bufferSize, replication, blockSize, @@ -122,8 +154,8 @@ public class HoodieWrapperFileSystem extends FileSystem { return fsDataOutputStream; } - SizeAwareFSDataOutputStream os = - new SizeAwareFSDataOutputStream(fsDataOutputStream, new Runnable() { + SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(fsDataOutputStream, + new Runnable() { @Override public void run() { openStreams.remove(path.getName()); @@ -160,14 +192,13 @@ public class HoodieWrapperFileSystem extends FileSystem { } @Override - public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) - throws IOException { + public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException { return fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize); } @Override - public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, - Progressable progress) throws IOException { + public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress) + throws IOException { return fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress); } @@ -175,14 +206,12 @@ public class HoodieWrapperFileSystem extends FileSystem { public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { return fileSystem - .create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, - progress); + .create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress); } @Override public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, - int bufferSize, short replication, long blockSize, Progressable progress) - throws IOException { + int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { return fileSystem .create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize, progress); @@ -197,7 +226,6 @@ public class HoodieWrapperFileSystem extends FileSystem { progress, checksumOpt); } - @Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException { @@ -205,7 +233,6 @@ public class HoodieWrapperFileSystem extends FileSystem { .create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize); } - @Override public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException { @@ -228,13 +255,13 @@ public class HoodieWrapperFileSystem extends FileSystem { } @Override - public void setWorkingDirectory(Path new_dir) { - fileSystem.setWorkingDirectory(convertToDefaultPath(new_dir)); + public Path getWorkingDirectory() { + return convertToHoodiePath(fileSystem.getWorkingDirectory()); } @Override - public Path getWorkingDirectory() { - return convertToHoodiePath(fileSystem.getWorkingDirectory()); + public void setWorkingDirectory(Path newDir) { + fileSystem.setWorkingDirectory(convertToDefaultPath(newDir)); } @Override @@ -290,8 +317,7 @@ public class HoodieWrapperFileSystem extends FileSystem { } @Override - public BlockLocation[] getFileBlockLocations(Path p, long start, long len) - throws IOException { + public BlockLocation[] getFileBlockLocations(Path p, long start, long len) throws IOException { return fileSystem.getFileBlockLocations(convertToDefaultPath(p), start, len); } @@ -319,17 +345,16 @@ public class HoodieWrapperFileSystem extends FileSystem { public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { return fileSystem - .createNonRecursive(convertToDefaultPath(f), overwrite, bufferSize, replication, - blockSize, progress); + .createNonRecursive(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, + progress); } @Override public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite, - int bufferSize, short replication, long blockSize, Progressable progress) - throws IOException { + int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { return fileSystem - .createNonRecursive(convertToDefaultPath(f), permission, overwrite, bufferSize, - replication, blockSize, progress); + .createNonRecursive(convertToDefaultPath(f), permission, overwrite, bufferSize, replication, + blockSize, progress); } @Override @@ -418,20 +443,17 @@ public class HoodieWrapperFileSystem extends FileSystem { } @Override - public FileStatus[] listStatus(Path f, PathFilter filter) - throws IOException { + public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException { return fileSystem.listStatus(convertToDefaultPath(f), filter); } @Override - public FileStatus[] listStatus(Path[] files) - throws IOException { + public FileStatus[] listStatus(Path[] files) throws IOException { return fileSystem.listStatus(convertDefaults(files)); } @Override - public FileStatus[] listStatus(Path[] files, PathFilter filter) - throws IOException { + public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException { return fileSystem.listStatus(convertDefaults(files), filter); } @@ -441,20 +463,17 @@ public class HoodieWrapperFileSystem extends FileSystem { } @Override - public FileStatus[] globStatus(Path pathPattern, PathFilter filter) - throws IOException { + public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException { return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter); } @Override - public RemoteIterator listLocatedStatus(Path f) - throws IOException { + public RemoteIterator listLocatedStatus(Path f) throws IOException { return fileSystem.listLocatedStatus(convertToDefaultPath(f)); } @Override - public RemoteIterator listFiles(Path f, boolean recursive) - throws IOException { + public RemoteIterator listFiles(Path f, boolean recursive) throws IOException { return fileSystem.listFiles(convertToDefaultPath(f), recursive); } @@ -498,8 +517,8 @@ public class HoodieWrapperFileSystem extends FileSystem { @Override public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst) throws IOException { - fileSystem.copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src), - convertToDefaultPath(dst)); + fileSystem + .copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src), convertToDefaultPath(dst)); } @Override @@ -525,15 +544,13 @@ public class HoodieWrapperFileSystem extends FileSystem { } @Override - public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) - throws IOException { - return convertToHoodiePath(fileSystem.startLocalOutput(convertToDefaultPath(fsOutputFile), - convertToDefaultPath(tmpLocalFile))); + public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException { + return convertToHoodiePath(fileSystem + .startLocalOutput(convertToDefaultPath(fsOutputFile), convertToDefaultPath(tmpLocalFile))); } @Override - public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) - throws IOException { + public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException { fileSystem.completeLocalOutput(convertToDefaultPath(fsOutputFile), convertToDefaultPath(tmpLocalFile)); } @@ -574,23 +591,18 @@ public class HoodieWrapperFileSystem extends FileSystem { } @Override - public void access(Path path, FsAction mode) - throws IOException { + public void access(Path path, FsAction mode) throws IOException { fileSystem.access(convertToDefaultPath(path), mode); } @Override - public void createSymlink(Path target, Path link, boolean createParent) - throws - IOException { + public void createSymlink(Path target, Path link, boolean createParent) throws IOException { fileSystem .createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent); } @Override - public FileStatus getFileLinkStatus(Path f) - throws - IOException { + public FileStatus getFileLinkStatus(Path f) throws IOException { return fileSystem.getFileLinkStatus(convertToDefaultPath(f)); } @@ -651,8 +663,7 @@ public class HoodieWrapperFileSystem extends FileSystem { @Override public Path createSnapshot(Path path, String snapshotName) throws IOException { - return convertToHoodiePath( - fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName)); + return convertToHoodiePath(fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName)); } @Override @@ -718,8 +729,7 @@ public class HoodieWrapperFileSystem extends FileSystem { } @Override - public Map getXAttrs(Path path, List names) - throws IOException { + public Map getXAttrs(Path path, List names) throws IOException { return fileSystem.getXAttrs(convertToDefaultPath(path), names); } @@ -734,13 +744,13 @@ public class HoodieWrapperFileSystem extends FileSystem { } @Override - public void setConf(Configuration conf) { - // ignore this. we will set conf on init + public Configuration getConf() { + return fileSystem.getConf(); } @Override - public Configuration getConf() { - return fileSystem.getConf(); + public void setConf(Configuration conf) { + // ignore this. we will set conf on init } @Override @@ -762,15 +772,6 @@ public class HoodieWrapperFileSystem extends FileSystem { return convertPathWithScheme(oldPath, getHoodieScheme(fileSystem.getScheme())); } - public static Path convertToHoodiePath(Path file, Configuration conf) { - try { - String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); - return convertPathWithScheme(file, getHoodieScheme(scheme)); - } catch (HoodieIOException e) { - throw e; - } - } - private Path convertToDefaultPath(Path oldPath) { return convertPathWithScheme(oldPath, fileSystem.getScheme()); } @@ -783,30 +784,6 @@ public class HoodieWrapperFileSystem extends FileSystem { return psrcsNew; } - private static Path convertPathWithScheme(Path oldPath, String newScheme) { - URI oldURI = oldPath.toUri(); - URI newURI; - try { - newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(), - oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment()); - return new Path(newURI); - } catch (URISyntaxException e) { - // TODO - Better Exception handling - throw new RuntimeException(e); - } - } - - public static String getHoodieScheme(String scheme) { - String newScheme; - if (SUPPORT_SCHEMES.contains(scheme)) { - newScheme = HOODIE_SCHEME_PREFIX + scheme; - } else { - throw new IllegalArgumentException( - "BlockAlignedAvroParquetWriter does not support scheme " + scheme); - } - return newScheme; - } - public long getBytesWritten(Path file) { if (openStreams.containsKey(file.getName())) { return openStreams.get(file.getName()).getBytesWritten(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java index fe275e159..78ef4960c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/HoodieMetrics.java @@ -30,14 +30,14 @@ import org.apache.log4j.Logger; */ public class HoodieMetrics { - private HoodieWriteConfig config = null; - private String tableName = null; private static Logger logger = LogManager.getLogger(HoodieMetrics.class); // Some timers public String rollbackTimerName = null; public String cleanTimerName = null; public String commitTimerName = null; public String finalizeTimerName = null; + private HoodieWriteConfig config = null; + private String tableName = null; private Timer rollbackTimer = null; private Timer cleanTimer = null; private Timer commitTimer = null; @@ -113,8 +113,9 @@ public class HoodieMetrics { public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) { if (config.isMetricsOn()) { - logger.info(String.format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)", - durationInMs, numFilesDeleted)); + logger.info(String + .format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)", durationInMs, + numFilesDeleted)); registerGauge(getMetricsName("rollback", "duration"), durationInMs); registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted); } @@ -122,8 +123,9 @@ public class HoodieMetrics { public void updateCleanMetrics(long durationInMs, int numFilesDeleted) { if (config.isMetricsOn()) { - logger.info(String.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)", - durationInMs, numFilesDeleted)); + logger.info(String + .format("Sending clean metrics (duration=%d, numFilesDeleted=%d)", durationInMs, + numFilesDeleted)); registerGauge(getMetricsName("clean", "duration"), durationInMs); registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted); } @@ -131,8 +133,9 @@ public class HoodieMetrics { public void updateFinalizeWriteMetrics(long durationInMs, int numFilesFinalized) { if (config.isMetricsOn()) { - logger.info(String.format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)", - durationInMs, numFilesFinalized)); + logger.info(String + .format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)", + durationInMs, numFilesFinalized)); registerGauge(getMetricsName("finalize", "duration"), durationInMs); registerGauge(getMetricsName("finalize", "numFilesFinalized"), numFilesFinalized); } @@ -140,8 +143,7 @@ public class HoodieMetrics { @VisibleForTesting String getMetricsName(String action, String metric) { - return config == null ? null : - String.format("%s.%s.%s", tableName, action, metric); + return config == null ? null : String.format("%s.%s.%s", tableName, action, metric); } void registerGauge(String metricName, final long value) { @@ -154,7 +156,8 @@ public class HoodieMetrics { } }); } catch (Exception e) { - // Here we catch all exception, so the major upsert pipeline will not be affected if the metrics system + // Here we catch all exception, so the major upsert pipeline will not be affected if the + // metrics system // has some issues. logger.error("Failed to send metrics: ", e); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java index c5a357317..924257493 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/Metrics.java @@ -40,7 +40,7 @@ public class Metrics { if (reporter == null) { throw new RuntimeException("Cannot initialize Reporter."); } -// reporter.start(); + // reporter.start(); Runtime.getRuntime().addShutdownHook(new Thread() { @Override diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsGraphiteReporter.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsGraphiteReporter.java index bc7d024a5..354693a89 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsGraphiteReporter.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsGraphiteReporter.java @@ -33,14 +33,13 @@ import org.apache.log4j.Logger; */ public class MetricsGraphiteReporter extends MetricsReporter { + private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class); private final MetricRegistry registry; private final GraphiteReporter graphiteReporter; private final HoodieWriteConfig config; private String serverHost; private int serverPort; - private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class); - public MetricsGraphiteReporter(HoodieWriteConfig config, MetricRegistry registry) { this.registry = registry; this.config = config; @@ -49,8 +48,8 @@ public class MetricsGraphiteReporter extends MetricsReporter { this.serverHost = config.getGraphiteServerHost(); this.serverPort = config.getGraphiteServerPort(); if (serverHost == null || serverPort == 0) { - throw new RuntimeException( - String.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].", + throw new RuntimeException(String + .format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].", serverHost, serverPort)); } @@ -81,14 +80,10 @@ public class MetricsGraphiteReporter extends MetricsReporter { } private GraphiteReporter createGraphiteReport() { - Graphite graphite = new Graphite( - new InetSocketAddress(serverHost, serverPort)); + Graphite graphite = new Graphite(new InetSocketAddress(serverHost, serverPort)); String reporterPrefix = config.getGraphiteMetricPrefix(); - return GraphiteReporter.forRegistry(registry) - .prefixedWith(reporterPrefix) - .convertRatesTo(TimeUnit.SECONDS) - .convertDurationsTo(TimeUnit.MILLISECONDS) - .filter(MetricFilter.ALL) - .build(graphite); + return GraphiteReporter.forRegistry(registry).prefixedWith(reporterPrefix) + .convertRatesTo(TimeUnit.SECONDS).convertDurationsTo(TimeUnit.MILLISECONDS) + .filter(MetricFilter.ALL).build(graphite); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterFactory.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterFactory.java index 31bc555c7..0d6e64fe1 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterFactory.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterFactory.java @@ -28,8 +28,7 @@ public class MetricsReporterFactory { private static Logger logger = LogManager.getLogger(MetricsReporterFactory.class); - public static MetricsReporter createReporter(HoodieWriteConfig config, - MetricRegistry registry) { + public static MetricsReporter createReporter(HoodieWriteConfig config, MetricRegistry registry) { MetricsReporterType type = config.getMetricsReporterType(); MetricsReporter reporter = null; switch (type) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterType.java b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterType.java index 46e128ddb..1b4667ded 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterType.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/metrics/MetricsReporterType.java @@ -21,6 +21,5 @@ package com.uber.hoodie.metrics; * future. */ public enum MetricsReporterType { - GRAPHITE, - INMEMORY + GRAPHITE, INMEMORY } 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 ea80630fa..db32a908d 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 @@ -75,23 +75,425 @@ import scala.Tuple2; /** * Implementation of a very heavily read-optimized Hoodie Table where - * + *

* INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing * file, to expand it - * + *

* UPDATES - Produce a new version of the file, just replacing the updated records with new values */ public class HoodieCopyOnWriteTable extends HoodieTable { + private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class); + public HoodieCopyOnWriteTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { super(config, metaClient); } - private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class); + private static PairFlatMapFunction>, String, + PartitionCleanStat> deleteFilesFunc( + HoodieTable table) { + return (PairFlatMapFunction>, String, PartitionCleanStat>) + iter -> { + Map partitionCleanStatMap = new HashMap<>(); + + FileSystem fs = table.getMetaClient().getFs(); + while (iter.hasNext()) { + Tuple2 partitionDelFileTuple = iter.next(); + String partitionPath = partitionDelFileTuple._1(); + String deletePathStr = partitionDelFileTuple._2(); + Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr); + if (!partitionCleanStatMap.containsKey(partitionPath)) { + partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath)); + } + PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath); + partitionCleanStat.addDeleteFilePatterns(deletePathStr); + partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult); + } + + return partitionCleanStatMap.entrySet().stream() + .map(e -> new Tuple2<>(e.getKey(), e.getValue())) + .collect(Collectors.toList()).iterator(); + }; + } + + private static PairFlatMapFunction getFilesToDeleteFunc(HoodieTable table, + HoodieWriteConfig config) { + return (PairFlatMapFunction) partitionPathToClean -> { + HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config); + return cleaner.getDeletePaths(partitionPathToClean).stream() + .map(deleteFile -> new Tuple2<>(partitionPathToClean, deleteFile.toString())).iterator(); + }; + } + + private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) + throws IOException { + Path deletePath = new Path(deletePathStr); + logger.debug("Working on delete path :" + deletePath); + boolean deleteResult = fs.delete(deletePath, false); + if (deleteResult) { + logger.debug("Cleaned file at path :" + deletePath); + } + return deleteResult; + } + + @Override + public Partitioner getUpsertPartitioner(WorkloadProfile profile) { + if (profile == null) { + throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); + } + return new UpsertPartitioner(profile); + } + + @Override + public Partitioner getInsertPartitioner(WorkloadProfile profile) { + return getUpsertPartitioner(profile); + } + + @Override + public boolean isWorkloadProfileNeeded() { + return true; + } + + @Override + public JavaRDD compact(JavaSparkContext jsc, String commitTime) { + throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); + } + + public Iterator> handleUpdate(String commitTime, String fileLoc, + Iterator> recordItr) throws IOException { + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr); + return handleUpdateInternal(upsertHandle, commitTime, fileLoc); + } + + public Iterator> handleUpdate(String commitTime, String fileLoc, + Map> keyToNewRecords) throws IOException { + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, keyToNewRecords); + return handleUpdateInternal(upsertHandle, commitTime, fileLoc); + } + + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, + String commitTime, String fileLoc) throws IOException { + if (upsertHandle.getOldFilePath() == null) { + throw new HoodieUpsertException( + "Error in finding the old file path at commit " + commitTime + " at fileLoc: " + fileLoc); + } else { + AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); + ParquetReader reader = AvroParquetReader.builder(upsertHandle.getOldFilePath()) + .withConf(getHadoopConf()).build(); + try { + IndexedRecord record; + while ((record = reader.read()) != null) { + // Two types of writes here (new record, and old record). + // We have already catch the exception during writing new records. + // But for old records, we should fail if any exception happens. + upsertHandle.write((GenericRecord) record); + } + } catch (IOException e) { + throw new HoodieUpsertException( + "Failed to read record from " + upsertHandle.getOldFilePath() + " with new Schema " + + upsertHandle.getSchema(), e); + } finally { + reader.close(); + upsertHandle.close(); + } + } + //TODO(vc): This needs to be revisited + if (upsertHandle.getWriteStatus().getPartitionPath() == null) { + logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " + + upsertHandle.getWriteStatus()); + } + return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())) + .iterator(); + } + + protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, + Iterator> recordItr) { + return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc); + } + + protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, + Map> keyToNewRecords) { + return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileLoc); + } + + public Iterator> handleInsert(String commitTime, + Iterator> recordItr) throws Exception { + return new LazyInsertIterable<>(recordItr, config, commitTime, this); + } + + @SuppressWarnings("unchecked") + @Override + public Iterator> handleUpsertPartition(String commitTime, Integer partition, + Iterator recordItr, Partitioner partitioner) { + UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner; + BucketInfo binfo = upsertPartitioner.getBucketInfo(partition); + BucketType btype = binfo.bucketType; + try { + if (btype.equals(BucketType.INSERT)) { + return handleInsert(commitTime, recordItr); + } else if (btype.equals(BucketType.UPDATE)) { + return handleUpdate(commitTime, binfo.fileLoc, recordItr); + } else { + throw new HoodieUpsertException( + "Unknown bucketType " + btype + " for partition :" + partition); + } + } catch (Throwable t) { + String msg = "Error upserting bucketType " + btype + " for partition :" + partition; + logger.error(msg, t); + throw new HoodieUpsertException(msg, t); + } + } + + @Override + public Iterator> handleInsertPartition(String commitTime, Integer partition, + Iterator recordItr, Partitioner partitioner) { + return handleUpsertPartition(commitTime, partition, recordItr, partitioner); + } + + /** + * Performs cleaning of partition paths according to cleaning policy and returns the number of + * files cleaned. Handles skews in partitions to clean by making files to clean as the unit of + * task distribution. + * + * @throws IllegalArgumentException if unknown cleaning policy is provided + */ + @Override + public List clean(JavaSparkContext jsc) { + try { + FileSystem fs = getMetaClient().getFs(); + List partitionsToClean = FSUtils + .getAllPartitionPaths(fs, getMetaClient().getBasePath(), + config.shouldAssumeDatePartitioning()); + logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config + .getCleanerPolicy()); + if (partitionsToClean.isEmpty()) { + logger.info("Nothing to clean here mom. It is already clean"); + return Collections.emptyList(); + } + return cleanPartitionPaths(partitionsToClean, jsc); + } catch (IOException e) { + throw new HoodieIOException("Failed to clean up after commit", e); + } + } + + /** + * Common method used for cleaning out parquet files under a partition path during rollback of a + * set of commits + */ + protected Map deleteCleanedFiles(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(); + for (FileStatus file : toBeDeleted) { + boolean success = fs.delete(file.getPath(), false); + results.put(file, success); + logger.info("Delete file " + file.getPath() + "\t" + success); + } + return results; + } + + @Override + public List rollback(JavaSparkContext jsc, List commits) + throws IOException { + String actionType = this.getCommitActionType(); + HoodieActiveTimeline activeTimeline = this.getActiveTimeline(); + List inflights = this.getInflightCommitTimeline().getInstants() + .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + + // Atomically unpublish all the commits + commits.stream().filter(s -> !inflights.contains(s)) + .map(s -> new HoodieInstant(false, actionType, s)) + .forEach(activeTimeline::revertToInflight); + logger.info("Unpublished " + commits); + + // delete all the data files for all these commits + logger.info("Clean out all parquet files generated for commits: " + commits); + List stats = jsc.parallelize(FSUtils + .getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(), + config.shouldAssumeDatePartitioning())) + .map((Function) partitionPath -> { + // Scan all partitions files with this commit time + Map results = deleteCleanedFiles(partitionPath, commits); + return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) + .withDeletedFileResults(results).build(); + }).collect(); + + // clean temporary data files + cleanTemporaryDataFiles(jsc); + + // Remove the rolled back inflight commits + commits.stream().map(s -> new HoodieInstant(true, actionType, s)) + .forEach(activeTimeline::deleteInflight); + logger.info("Deleted inflight commits " + commits); + return stats; + } + + /** + * Finalize the written data files + * + * @param writeStatuses List of WriteStatus + * @return number of files finalized + */ + @Override + @SuppressWarnings("unchecked") + public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { + if (!config.shouldUseTempFolderForCopyOnWrite()) { + return Optional.empty(); + } + + // This is to rename each data file from temporary path to its final location + List> results = jsc + .parallelize(writeStatuses, config.getFinalizeWriteParallelism()).map(writeStatus -> { + Tuple2 writeStatTuple2 = (Tuple2) + writeStatus; + HoodieWriteStat writeStat = writeStatTuple2._2(); + final FileSystem fs = getMetaClient().getFs(); + final Path finalPath = new Path(config.getBasePath(), writeStat.getPath()); + + if (writeStat.getTempPath() != null) { + final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath()); + boolean success; + try { + logger.info("Renaming temporary file: " + tempPath + " to " + finalPath); + success = fs.rename(tempPath, finalPath); + } catch (IOException e) { + throw new HoodieIOException( + "Failed to rename file: " + tempPath + " to " + finalPath); + } + + if (!success) { + throw new HoodieIOException( + "Failed to rename file: " + tempPath + " to " + finalPath); + } + } + + return new Tuple2<>(writeStat.getPath(), true); + }).collect(); + + // clean temporary data files + cleanTemporaryDataFiles(jsc); + + return Optional.of(results.size()); + } + + /** + * Clean temporary data files that are produced from previous failed commit or retried spark + * stages. + */ + private void cleanTemporaryDataFiles(JavaSparkContext jsc) { + if (!config.shouldUseTempFolderForCopyOnWrite()) { + return; + } + + final FileSystem fs = getMetaClient().getFs(); + final Path temporaryFolder = new Path(config.getBasePath(), + HoodieTableMetaClient.TEMPFOLDER_NAME); + try { + if (!fs.exists(temporaryFolder)) { + logger.info("Temporary folder does not exist: " + temporaryFolder); + return; + } + List fileStatusesList = Arrays.asList(fs.listStatus(temporaryFolder)); + List> results = jsc + .parallelize(fileStatusesList, config.getFinalizeWriteParallelism()).map(fileStatus -> { + FileSystem fs1 = getMetaClient().getFs(); + boolean success = fs1.delete(fileStatus.getPath(), false); + logger + .info("Deleting file in temporary folder" + fileStatus.getPath() + "\t" + success); + return new Tuple2<>(fileStatus.getPath().toString(), success); + }).collect(); + + for (Tuple2 result : results) { + if (!result._2()) { + logger.info("Failed to delete file: " + result._1()); + throw new HoodieIOException("Failed to delete file in temporary folder: " + result._1()); + } + } + } catch (IOException e) { + throw new HoodieIOException( + "Failed to clean data files in temporary folder: " + temporaryFolder); + } + } + + private List cleanPartitionPaths(List partitionsToClean, + JavaSparkContext jsc) { + int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism()); + logger.info("Using cleanerParallelism: " + cleanerParallelism); + List> partitionCleanStats = jsc + .parallelize(partitionsToClean, cleanerParallelism) + .flatMapToPair(getFilesToDeleteFunc(this, config)) + .repartition(cleanerParallelism) // repartition to remove skews + .mapPartitionsToPair(deleteFilesFunc(this)).reduceByKey( + // merge partition level clean stats below + (Function2) (e1, e2) -> e1 + .merge(e2)).collect(); + + Map partitionCleanStatsMap = partitionCleanStats.stream() + .collect(Collectors.toMap(e -> e._1(), e -> e._2())); + + HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config); + // Return PartitionCleanStat for each partition passed. + return partitionsToClean.stream().map(partitionPath -> { + PartitionCleanStat partitionCleanStat = + (partitionCleanStatsMap.containsKey(partitionPath)) ? partitionCleanStatsMap + .get(partitionPath) : new PartitionCleanStat(partitionPath); + return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()) + .withPartitionPath(partitionPath) + .withEarliestCommitRetained(cleaner.getEarliestCommitToRetain()) + .withDeletePathPattern(partitionCleanStat.deletePathPatterns) + .withSuccessfulDeletes(partitionCleanStat.successDeleteFiles) + .withFailedDeletes(partitionCleanStat.failedDeleteFiles).build(); + }).collect(Collectors.toList()); + } enum BucketType { - UPDATE, - INSERT + UPDATE, INSERT + } + + private static class PartitionCleanStat implements Serializable { + + private final String partitionPath; + private final List deletePathPatterns = new ArrayList<>(); + private final List successDeleteFiles = new ArrayList<>(); + private final List failedDeleteFiles = new ArrayList<>(); + + private PartitionCleanStat(String partitionPath) { + this.partitionPath = partitionPath; + } + + private void addDeletedFileResult(String deletePathStr, Boolean deletedFileResult) { + if (deletedFileResult) { + successDeleteFiles.add(deletePathStr); + } else { + failedDeleteFiles.add(deletePathStr); + } + } + + private void addDeleteFilePatterns(String deletePathStr) { + deletePathPatterns.add(deletePathStr); + } + + private PartitionCleanStat merge(PartitionCleanStat other) { + if (!this.partitionPath.equals(other.partitionPath)) { + throw new RuntimeException(String + .format("partitionPath is not a match: (%s, %s)", partitionPath, other.partitionPath)); + } + successDeleteFiles.addAll(other.successDeleteFiles); + deletePathPatterns.addAll(other.deletePathPatterns); + failedDeleteFiles.addAll(other.failedDeleteFiles); + return this; + } } /** @@ -150,45 +552,37 @@ public class HoodieCopyOnWriteTable extends Hoodi } } - /** * Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition) */ class UpsertPartitioner extends Partitioner { + /** + * List of all small files to be corrected + */ + List smallFiles = new ArrayList(); /** * Total number of RDD partitions, is determined by total buckets we want to pack the incoming * workload into */ private int totalBuckets = 0; - /** * Stat for the current workload. Helps in determining total inserts, upserts etc. */ private WorkloadStat globalStat; - /** * Helps decide which bucket an incoming update should go to. */ private HashMap updateLocationToBucket; - - /** * Helps us pack inserts into 1 or more buckets depending on number of incoming records. */ private HashMap> partitionPathToInsertBuckets; - - /** * Remembers what type each bucket is for later. */ private HashMap bucketInfoMap; - /** - * List of all small files to be corrected - */ - List smallFiles = new ArrayList(); - UpsertPartitioner(WorkloadProfile profile) { updateLocationToBucket = new HashMap<>(); partitionPathToInsertBuckets = new HashMap<>(); @@ -198,16 +592,17 @@ public class HoodieCopyOnWriteTable extends Hoodi assignUpdates(profile); assignInserts(profile); - logger.info("Total Buckets :" + totalBuckets + ", " + - "buckets info => " + bucketInfoMap + ", \n" + - "Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n" + - "UpdateLocations mapped to buckets =>" + updateLocationToBucket); + logger.info( + "Total Buckets :" + totalBuckets + ", " + "buckets info => " + bucketInfoMap + ", \n" + + "Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n" + + "UpdateLocations mapped to buckets =>" + updateLocationToBucket); } private void assignUpdates(WorkloadProfile profile) { // each update location gets a partition WorkloadStat gStat = profile.getGlobalStat(); - for (Map.Entry> updateLocEntry : gStat.getUpdateLocationToCount().entrySet()) { + for (Map.Entry> updateLocEntry : gStat.getUpdateLocationToCount() + .entrySet()) { addUpdateBucket(updateLocEntry.getKey()); } } @@ -270,10 +665,10 @@ public class HoodieCopyOnWriteTable extends Hoodi } int insertBuckets = (int) Math.max(totalUnassignedInserts / insertRecordsPerBucket, 1L); - logger - .info("After small file assignment: unassignedInserts => " + totalUnassignedInserts - + ", totalInsertBuckets => " + insertBuckets - + ", recordsPerBucket => " + insertRecordsPerBucket); + logger.info( + "After small file assignment: unassignedInserts => " + totalUnassignedInserts + + ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => " + + insertRecordsPerBucket); for (int b = 0; b < insertBuckets; b++) { bucketNumbers.add(totalBuckets); recordsPerBucket.add(totalUnassignedInserts / insertBuckets); @@ -339,8 +734,8 @@ public class HoodieCopyOnWriteTable extends Hoodi */ private long averageBytesPerRecord() { long avgSize = 0L; - HoodieTimeline commitTimeline = - metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(); + HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getCommitTimeline() + .filterCompletedInstants(); try { if (!commitTimeline.empty()) { HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); @@ -372,7 +767,8 @@ public class HoodieCopyOnWriteTable extends Hoodi @Override public int getPartition(Object key) { - Tuple2> keyLocation = (Tuple2>) key; + Tuple2> keyLocation = (Tuple2>) key; if (keyLocation._2().isDefined()) { HoodieRecordLocation location = keyLocation._2().get(); return updateLocationToBucket.get(location.getFileId()); @@ -396,420 +792,4 @@ public class HoodieCopyOnWriteTable extends Hoodi } } } - - - @Override - public Partitioner getUpsertPartitioner(WorkloadProfile profile) { - if (profile == null) { - throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); - } - return new UpsertPartitioner(profile); - } - - @Override - public Partitioner getInsertPartitioner(WorkloadProfile profile) { - return getUpsertPartitioner(profile); - } - - @Override - public boolean isWorkloadProfileNeeded() { - return true; - } - - @Override - public JavaRDD compact(JavaSparkContext jsc, String commitTime) { - throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); - } - - - public Iterator> handleUpdate(String commitTime, String fileLoc, - Iterator> recordItr) - throws IOException { - // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr); - return handleUpdateInternal(upsertHandle, commitTime, fileLoc); - } - - public Iterator> handleUpdate(String commitTime, String fileLoc, - Map> keyToNewRecords) - throws IOException { - // these are updates - HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, keyToNewRecords); - return handleUpdateInternal(upsertHandle, commitTime, fileLoc); - } - - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String commitTime, String fileLoc) - throws IOException { - if (upsertHandle.getOldFilePath() == null) { - throw new HoodieUpsertException("Error in finding the old file path at commit " + - commitTime + " at fileLoc: " + fileLoc); - } else { - AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema()); - ParquetReader reader = - AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(getHadoopConf()) - .build(); - try { - IndexedRecord record; - while ((record = reader.read()) != null) { - // Two types of writes here (new record, and old record). - // We have already catch the exception during writing new records. - // But for old records, we should fail if any exception happens. - upsertHandle.write((GenericRecord) record); - } - } catch (IOException e) { - throw new HoodieUpsertException( - "Failed to read record from " + upsertHandle.getOldFilePath() - + " with new Schema " + upsertHandle.getSchema(), e); - } finally { - reader.close(); - upsertHandle.close(); - } - } - //TODO(vc): This needs to be revisited - if (upsertHandle.getWriteStatus().getPartitionPath() == null) { - logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() - + ", " + upsertHandle.getWriteStatus()); - } - return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())) - .iterator(); - } - - protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, - Iterator> recordItr) { - return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc); - } - - protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc, - Map> keyToNewRecords) { - return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileLoc); - } - - public Iterator> handleInsert(String commitTime, - Iterator> recordItr) throws Exception { - return new LazyInsertIterable<>(recordItr, config, commitTime, this); - } - - - @SuppressWarnings("unchecked") - @Override - public Iterator> handleUpsertPartition(String commitTime, Integer partition, - Iterator recordItr, Partitioner partitioner) { - UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner; - BucketInfo binfo = upsertPartitioner.getBucketInfo(partition); - BucketType btype = binfo.bucketType; - try { - if (btype.equals(BucketType.INSERT)) { - return handleInsert(commitTime, recordItr); - } else if (btype.equals(BucketType.UPDATE)) { - return handleUpdate(commitTime, binfo.fileLoc, recordItr); - } else { - throw new HoodieUpsertException( - "Unknown bucketType " + btype + " for partition :" + partition); - } - } catch (Throwable t) { - String msg = "Error upserting bucketType " + btype + " for partition :" + partition; - logger.error(msg, t); - throw new HoodieUpsertException(msg, t); - } - } - - @Override - public Iterator> handleInsertPartition(String commitTime, Integer partition, - Iterator recordItr, - Partitioner partitioner) { - return handleUpsertPartition(commitTime, partition, recordItr, partitioner); - } - - /** - * Performs cleaning of partition paths according to cleaning policy and returns the number of - * files cleaned. Handles skews in partitions to clean by making files to clean as the unit of - * task distribution. - * - * @throws IllegalArgumentException if unknown cleaning policy is provided - */ - @Override - public List clean(JavaSparkContext jsc) { - try { - FileSystem fs = getMetaClient().getFs(); - List partitionsToClean = - FSUtils.getAllPartitionPaths(fs, getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); - logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config - .getCleanerPolicy()); - if (partitionsToClean.isEmpty()) { - logger.info("Nothing to clean here mom. It is already clean"); - return Collections.emptyList(); - } - return cleanPartitionPaths(partitionsToClean, jsc); - } catch (IOException e) { - throw new HoodieIOException("Failed to clean up after commit", e); - } - } - - /** - * Common method used for cleaning out parquet files under a partition path during rollback of a - * set of commits - */ - protected Map deleteCleanedFiles(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(); - for (FileStatus file : toBeDeleted) { - boolean success = fs.delete(file.getPath(), false); - results.put(file, success); - logger.info("Delete file " + file.getPath() + "\t" + success); - } - return results; - } - - @Override - public List rollback(JavaSparkContext jsc, List commits) - throws IOException { - String actionType = this.getCommitActionType(); - HoodieActiveTimeline activeTimeline = this.getActiveTimeline(); - List inflights = this.getInflightCommitTimeline().getInstants() - .map(HoodieInstant::getTimestamp) - .collect(Collectors.toList()); - - // Atomically unpublish all the commits - commits.stream().filter(s -> !inflights.contains(s)) - .map(s -> new HoodieInstant(false, actionType, s)) - .forEach(activeTimeline::revertToInflight); - logger.info("Unpublished " + commits); - - // delete all the data files for all these commits - logger.info("Clean out all parquet files generated for commits: " + commits); - List stats = jsc.parallelize( - FSUtils.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning())) - .map((Function) partitionPath -> { - // Scan all partitions files with this commit time - Map results = deleteCleanedFiles(partitionPath, commits); - return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) - .withDeletedFileResults(results).build(); - }).collect(); - - // clean temporary data files - cleanTemporaryDataFiles(jsc); - - // Remove the rolled back inflight commits - commits.stream().map(s -> new HoodieInstant(true, actionType, s)) - .forEach(activeTimeline::deleteInflight); - logger.info("Deleted inflight commits " + commits); - return stats; - } - - /** - * Finalize the written data files - * - * @param writeStatuses List of WriteStatus - * @return number of files finalized - */ - @Override - @SuppressWarnings("unchecked") - public Optional finalizeWrite(JavaSparkContext jsc, List writeStatuses) { - if (!config.shouldUseTempFolderForCopyOnWrite()) { - return Optional.empty(); - } - - // This is to rename each data file from temporary path to its final location - List> results = jsc.parallelize(writeStatuses, config.getFinalizeWriteParallelism()) - .map(writeStatus -> { - Tuple2 writeStatTuple2 = (Tuple2) writeStatus; - HoodieWriteStat writeStat = writeStatTuple2._2(); - final FileSystem fs = getMetaClient().getFs(); - final Path finalPath = new Path(config.getBasePath(), writeStat.getPath()); - - if (writeStat.getTempPath() != null) { - final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath()); - boolean success; - try { - logger.info("Renaming temporary file: " + tempPath + " to " + finalPath); - success = fs.rename(tempPath, finalPath); - } catch (IOException e) { - throw new HoodieIOException("Failed to rename file: " + tempPath + " to " + finalPath); - } - - if (!success) { - throw new HoodieIOException("Failed to rename file: " + tempPath + " to " + finalPath); - } - } - - return new Tuple2<>(writeStat.getPath(), true); - }).collect(); - - // clean temporary data files - cleanTemporaryDataFiles(jsc); - - return Optional.of(results.size()); - } - - /** - * Clean temporary data files that are produced from previous failed commit or retried spark - * stages. - */ - private void cleanTemporaryDataFiles(JavaSparkContext jsc) { - if (!config.shouldUseTempFolderForCopyOnWrite()) { - return; - } - - final FileSystem fs = getMetaClient().getFs(); - final Path temporaryFolder = new Path(config.getBasePath(), - HoodieTableMetaClient.TEMPFOLDER_NAME); - try { - if (!fs.exists(temporaryFolder)) { - logger.info("Temporary folder does not exist: " + temporaryFolder); - return; - } - List fileStatusesList = Arrays.asList(fs.listStatus(temporaryFolder)); - List> results = jsc - .parallelize(fileStatusesList, config.getFinalizeWriteParallelism()) - .map(fileStatus -> { - FileSystem fs1 = getMetaClient().getFs(); - boolean success = fs1.delete(fileStatus.getPath(), false); - logger.info("Deleting file in temporary folder" + fileStatus.getPath() + "\t" - + success); - return new Tuple2<>(fileStatus.getPath().toString(), success); - }).collect(); - - for (Tuple2 result : results) { - if (!result._2()) { - logger.info("Failed to delete file: " + result._1()); - throw new HoodieIOException( - "Failed to delete file in temporary folder: " + result._1()); - } - } - } catch (IOException e) { - throw new HoodieIOException( - "Failed to clean data files in temporary folder: " + temporaryFolder); - } - } - - private static class PartitionCleanStat implements Serializable { - - private final String partitionPath; - private final List deletePathPatterns = new ArrayList<>(); - private final List successDeleteFiles = new ArrayList<>(); - private final List failedDeleteFiles = new ArrayList<>(); - - private PartitionCleanStat(String partitionPath) { - this.partitionPath = partitionPath; - } - - private void addDeletedFileResult(String deletePathStr, Boolean deletedFileResult) { - if (deletedFileResult) { - successDeleteFiles.add(deletePathStr); - } else { - failedDeleteFiles.add(deletePathStr); - } - } - - private void addDeleteFilePatterns(String deletePathStr) { - deletePathPatterns.add(deletePathStr); - } - - private PartitionCleanStat merge(PartitionCleanStat other) { - if (!this.partitionPath.equals(other.partitionPath)) { - throw new RuntimeException(String.format( - "partitionPath is not a match: (%s, %s)", - partitionPath, other.partitionPath)); - } - successDeleteFiles.addAll(other.successDeleteFiles); - deletePathPatterns.addAll(other.deletePathPatterns); - failedDeleteFiles.addAll(other.failedDeleteFiles); - return this; - } - } - - private List cleanPartitionPaths(List partitionsToClean, - JavaSparkContext jsc) { - int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism()); - logger.info("Using cleanerParallelism: " + cleanerParallelism); - List> partitionCleanStats = jsc - .parallelize(partitionsToClean, cleanerParallelism) - .flatMapToPair(getFilesToDeleteFunc(this, config)) - .repartition(cleanerParallelism) // repartition to remove skews - .mapPartitionsToPair(deleteFilesFunc(this)) - .reduceByKey( - // merge partition level clean stats below - (Function2) (e1, e2) -> e1 - .merge(e2)) - .collect(); - - Map partitionCleanStatsMap = partitionCleanStats - .stream().collect(Collectors.toMap(e -> e._1(), e -> e._2())); - - HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config); - // Return PartitionCleanStat for each partition passed. - return partitionsToClean.stream().map(partitionPath -> { - PartitionCleanStat partitionCleanStat = - (partitionCleanStatsMap.containsKey(partitionPath)) ? - partitionCleanStatsMap.get(partitionPath) - : new PartitionCleanStat(partitionPath); - return HoodieCleanStat.newBuilder() - .withPolicy(config.getCleanerPolicy()) - .withPartitionPath(partitionPath) - .withEarliestCommitRetained(cleaner.getEarliestCommitToRetain()) - .withDeletePathPattern(partitionCleanStat.deletePathPatterns) - .withSuccessfulDeletes(partitionCleanStat.successDeleteFiles) - .withFailedDeletes(partitionCleanStat.failedDeleteFiles) - .build(); - }).collect(Collectors.toList()); - } - - private static PairFlatMapFunction>, String, PartitionCleanStat> deleteFilesFunc( - HoodieTable table) { - return (PairFlatMapFunction>, String, PartitionCleanStat>) iter -> { - Map partitionCleanStatMap = new HashMap<>(); - - FileSystem fs = table.getMetaClient().getFs(); - while (iter.hasNext()) { - Tuple2 partitionDelFileTuple = iter.next(); - String partitionPath = partitionDelFileTuple._1(); - String deletePathStr = partitionDelFileTuple._2(); - Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr); - if (!partitionCleanStatMap.containsKey(partitionPath)) { - partitionCleanStatMap.put(partitionPath, - new PartitionCleanStat(partitionPath)); - } - PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath); - partitionCleanStat.addDeleteFilePatterns(deletePathStr); - partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult); - } - - return partitionCleanStatMap.entrySet().stream() - .map(e -> new Tuple2<>(e.getKey(), e.getValue())) - .collect(Collectors.toList()).iterator(); - }; - } - - private static PairFlatMapFunction getFilesToDeleteFunc( - HoodieTable table, HoodieWriteConfig config) { - return (PairFlatMapFunction) partitionPathToClean -> { - HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config); - return cleaner.getDeletePaths(partitionPathToClean).stream() - .map(deleteFile -> new Tuple2<>(partitionPathToClean, deleteFile.toString())) - .iterator(); - }; - } - - private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) - throws IOException { - Path deletePath = new Path(deletePathStr); - logger.debug("Working on delete path :" + deletePath); - boolean deleteResult = fs.delete(deletePath, false); - if (deleteResult) { - logger.debug("Cleaned file at path :" + deletePath); - } - return deleteResult; - } } 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 078747774..79239ada8 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 @@ -62,19 +62,12 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; /** - * Implementation of a more real-time read-optimized Hoodie Table where - *

- * INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) - * Merge with the smallest existing file, to expand it - *

- *

- * UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the - * log file into the base file. - *

- *

- * WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an - * attempted commit action - *

+ * Implementation of a more real-time read-optimized Hoodie Table where

INSERTS - Same as + * HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the + * smallest existing file, to expand it

UPDATES - Appends the changes to a rolling log file + * maintained per file Id. Compaction merges the log file into the base file.

WARNING - MOR + * table type does not support nested rollbacks, every rollback must be followed by an attempted + * commit action

*/ public class HoodieMergeOnReadTable extends HoodieCopyOnWriteTable { @@ -88,57 +81,6 @@ public class HoodieMergeOnReadTable extends super(config, metaClient); } - /** - * 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. - */ - class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner { - - MergeOnReadUpsertPartitioner(WorkloadProfile profile) { - super(profile); - } - - @Override - protected List getSmallFiles(String partitionPath) { - - // smallFiles only for partitionPath - List smallFileLocations = new ArrayList<>(); - - // Init here since this class (and member variables) might not have been initialized - HoodieTimeline commitTimeline = getCompletedCommitTimeline(); - - if (!commitTimeline.empty()) { - HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); - // find smallest file in partition and append to it - 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(); - 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); - } - } - - return smallFileLocations; - } - - public List getSmallFileIds() { - return (List) smallFiles.stream().map(smallFile -> ((SmallFile) smallFile).location.getFileId()) - .collect(Collectors.toList()); - } - } - @Override public Partitioner getUpsertPartitioner(WorkloadProfile profile) { if (profile == null) { @@ -150,15 +92,16 @@ public class HoodieMergeOnReadTable extends @Override public Iterator> handleUpdate(String commitTime, String fileId, - Iterator> recordItr) throws IOException { + Iterator> recordItr) throws IOException { logger.info("Merging updates for commit " + commitTime + " for file " + fileId); - if(mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { - logger.info("Small file corrections for updates for commit " + commitTime + " for file " + fileId); + if (mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) { + logger.info( + "Small file corrections for updates for commit " + commitTime + " for file " + fileId); return super.handleUpdate(commitTime, fileId, recordItr); } else { - HoodieAppendHandle appendHandle = - new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr); + HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, commitTime, this, + fileId, recordItr); appendHandle.doAppend(); appendHandle.close(); return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus())) @@ -202,13 +145,11 @@ public class HoodieMergeOnReadTable extends if (commits.size() > 1) { throw new UnsupportedOperationException("Nested Rollbacks are not supported"); } - Map commitsAndCompactions = - this.getActiveTimeline() - .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, - HoodieActiveTimeline.DELTA_COMMIT_ACTION)) - .getInstants() - .filter(i -> commits.contains(i.getTimestamp())) - .collect(Collectors.toMap(i -> i.getTimestamp(), i -> i)); + Map commitsAndCompactions = this.getActiveTimeline() + .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, + HoodieActiveTimeline.DELTA_COMMIT_ACTION)).getInstants() + .filter(i -> commits.contains(i.getTimestamp())) + .collect(Collectors.toMap(i -> i.getTimestamp(), i -> i)); // Atomically un-publish all non-inflight commits commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue()) @@ -218,9 +159,9 @@ public class HoodieMergeOnReadTable extends Long startTime = System.currentTimeMillis(); - List allRollbackStats = jsc.parallelize - (FSUtils.getAllPartitionPaths(this.metaClient.getFs(), - this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) + List allRollbackStats = jsc.parallelize(FSUtils + .getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(), + config.shouldAssumeDatePartitioning())) .map((Function>) partitionPath -> { return commits.stream().map(commit -> { HoodieInstant instant = commitsAndCompactions.get(commit); @@ -228,56 +169,63 @@ public class HoodieMergeOnReadTable extends switch (instant.getAction()) { case HoodieTimeline.COMMIT_ACTION: try { - Map results = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit)); - hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) - .withDeletedFileResults(results).build(); + Map results = super + .deleteCleanedFiles(partitionPath, Arrays.asList(commit)); + hoodieRollbackStats = HoodieRollbackStat.newBuilder() + .withPartitionPath(partitionPath).withDeletedFileResults(results).build(); break; } catch (IOException io) { throw new UncheckedIOException("Failed to rollback for commit " + commit, io); } case HoodieTimeline.DELTA_COMMIT_ACTION: try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get()); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + this.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<>(); 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) + // 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)); // append rollback blocks for updates - if(commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { + if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) { commitMetadata.getPartitionToWriteStats().get(partitionPath).stream() .filter(wStat -> { - return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT + return wStat != null + && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null; - }) - .forEach(wStat -> { + }).forEach(wStat -> { HoodieLogFormat.Writer writer = null; try { - writer = HoodieLogFormat.newWriterBuilder() - .onParentPath(new Path(this.getMetaClient().getBasePath(), partitionPath)) + writer = HoodieLogFormat.newWriterBuilder().onParentPath( + new Path(this.getMetaClient().getBasePath(), partitionPath)) .withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit()) .withFs(this.metaClient.getFs()) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); Long numRollbackBlocks = 0L; // generate metadata - Map header = Maps.newHashMap(); + Map header = + Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); - header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, commit); - header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, - String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, + commit); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String + .valueOf( + HoodieCommandBlock.HoodieCommandBlockTypeEnum + .ROLLBACK_PREVIOUS_BLOCK + .ordinal())); // if update belongs to an existing log file - writer = writer.appendBlock(new HoodieCommandBlock( - header)); + writer = writer.appendBlock(new HoodieCommandBlock(header)); numRollbackBlocks++; - filesToNumBlocksRollback - .put(this.getMetaClient().getFs().getFileStatus(writer.getLogFile().getPath()), - 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); @@ -289,7 +237,8 @@ public class HoodieMergeOnReadTable extends } } }); - hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath) + hoodieRollbackStats = HoodieRollbackStat.newBuilder() + .withPartitionPath(partitionPath) .withDeletedFileResults(filesToDeletedStatus) .withRollbackBlockAppendResults(filesToNumBlocksRollback).build(); } @@ -297,17 +246,19 @@ public class HoodieMergeOnReadTable extends } catch (IOException io) { throw new UncheckedIOException("Failed to rollback for commit " + commit, io); } + default: + break; } return hoodieRollbackStats; }).collect(Collectors.toList()); }).flatMap(x -> x.iterator()).filter(x -> x != null).collect(); - commitsAndCompactions.entrySet().stream() - .map(entry -> new HoodieInstant(true, entry.getValue().getAction(), - entry.getValue().getTimestamp())) - .forEach(this.getActiveTimeline()::deleteInflight); + commitsAndCompactions.entrySet().stream().map( + entry -> new HoodieInstant(true, entry.getValue().getAction(), + entry.getValue().getTimestamp())).forEach(this.getActiveTimeline()::deleteInflight); - logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime)); + logger + .debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime)); return allRollbackStats; } @@ -317,4 +268,56 @@ public class HoodieMergeOnReadTable extends // do nothing for MOR tables return Optional.empty(); } + + /** + * 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. + */ + class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner { + + MergeOnReadUpsertPartitioner(WorkloadProfile profile) { + super(profile); + } + + @Override + protected List getSmallFiles(String partitionPath) { + + // smallFiles only for partitionPath + List smallFileLocations = new ArrayList<>(); + + // Init here since this class (and member variables) might not have been initialized + HoodieTimeline commitTimeline = getCompletedCommitTimeline(); + + if (!commitTimeline.empty()) { + HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); + // find smallest file in partition and append to it + 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(); + 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); + } + } + + return smallFileLocations; + } + + public List getSmallFileIds() { + return (List) smallFiles.stream() + .map(smallFile -> ((SmallFile) smallFile).location.getFileId()) + .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 728fc560a..6f62533c3 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 @@ -60,18 +60,28 @@ public abstract class HoodieTable implements Seri this.metaClient = metaClient; } + public static HoodieTable getHoodieTable( + HoodieTableMetaClient metaClient, HoodieWriteConfig config) { + switch (metaClient.getTableType()) { + case COPY_ON_WRITE: + return new HoodieCopyOnWriteTable<>(config, metaClient); + case MERGE_ON_READ: + return new HoodieMergeOnReadTable<>(config, metaClient); + default: + throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); + } + } + /** * Provides a partitioner to perform the upsert operation, based on the workload profile */ public abstract Partitioner getUpsertPartitioner(WorkloadProfile profile); - /** * Provides a partitioner to perform the insert operation, based on the workload profile */ public abstract Partitioner getInsertPartitioner(WorkloadProfile profile); - /** * Return whether this HoodieTable implementation can benefit from workload profiling */ @@ -131,7 +141,6 @@ public abstract class HoodieTable implements Seri return getCommitsTimeline().filterInflights(); } - /** * Get only the completed (no-inflights) clean timeline */ @@ -162,12 +171,12 @@ public abstract class HoodieTable implements Seri throw new HoodieSavepointException( "Could not get data files for savepoint " + savepointTime + ". No such savepoint."); } - HoodieInstant instant = - new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); + HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, + savepointTime); HoodieSavepointMetadata metadata = null; try { - metadata = AvroUtils.deserializeHoodieSavepointMetadata( - getActiveTimeline().getInstantDetails(instant).get()); + metadata = AvroUtils + .deserializeHoodieSavepointMetadata(getActiveTimeline().getInstantDetails(instant).get()); } catch (IOException e) { throw new HoodieSavepointException( "Could not get savepointed data files for savepoint " + savepointTime, e); @@ -189,7 +198,8 @@ public abstract class HoodieTable implements Seri 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 + // 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()); @@ -219,9 +229,10 @@ public abstract class HoodieTable implements Seri 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()); } - throw new HoodieCommitException( - "Could not commit on unknown storage type " + metaClient.getTableType()); } /** @@ -236,21 +247,9 @@ public abstract class HoodieTable implements Seri public abstract Iterator> handleInsertPartition(String commitTime, Integer partition, Iterator> recordIterator, Partitioner partitioner); - public static HoodieTable getHoodieTable( - HoodieTableMetaClient metaClient, HoodieWriteConfig config) { - switch (metaClient.getTableType()) { - case COPY_ON_WRITE: - return new HoodieCopyOnWriteTable<>(config, metaClient); - case MERGE_ON_READ: - return new HoodieMergeOnReadTable<>(config, metaClient); - default: - throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); - } - } - /** - * Run Compaction on the table. - * Compaction arranges the data so that it is optimized for data access + * Run Compaction on the table. Compaction arranges the data so that it is optimized for data + * access */ public abstract JavaRDD compact(JavaSparkContext jsc, String commitTime); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/UserDefinedBulkInsertPartitioner.java b/hoodie-client/src/main/java/com/uber/hoodie/table/UserDefinedBulkInsertPartitioner.java index 2ca51a31f..6adb8a982 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/UserDefinedBulkInsertPartitioner.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/UserDefinedBulkInsertPartitioner.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.table; import com.uber.hoodie.common.model.HoodieRecord; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java index 07e863690..ed5431f48 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java @@ -16,7 +16,6 @@ package com.uber.hoodie.table; - import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; @@ -31,7 +30,7 @@ import scala.Tuple2; /** * Information about incoming records for upsert/insert obtained either via sampling or * introspecting the data fully - * + *

* TODO(vc): Think about obtaining this directly from index.tagLocation */ public class WorkloadProfile implements Serializable { @@ -60,11 +59,9 @@ public class WorkloadProfile implements Serializa private void buildProfile() { Map>, Long> partitionLocationCounts = taggedRecords - .mapToPair(record -> - new Tuple2<>( - new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())), - record)) - .countByKey(); + .mapToPair(record -> new Tuple2<>( + new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())), + record)).countByKey(); for (Map.Entry>, Long> e : partitionLocationCounts .entrySet()) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java index 33764fa59..a93ac55af 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadStat.java @@ -17,10 +17,9 @@ package com.uber.hoodie.table; import com.uber.hoodie.common.model.HoodieRecordLocation; -import org.apache.commons.lang3.tuple.Pair; - import java.io.Serializable; import java.util.HashMap; +import org.apache.commons.lang3.tuple.Pair; /** * Wraps stats about a single partition path. diff --git a/hoodie-client/src/test/java/HoodieClientExample.java b/hoodie-client/src/test/java/HoodieClientExample.java index 166bd22de..ab81e2aca 100644 --- a/hoodie-client/src/test/java/HoodieClientExample.java +++ b/hoodie-client/src/test/java/HoodieClientExample.java @@ -14,7 +14,6 @@ * limitations under the License. */ - import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import com.uber.hoodie.HoodieWriteClient; @@ -38,24 +37,19 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; /** - * Driver program that uses the Hoodie client with synthetic workload, and performs basic - * operations.

+ * Driver program that uses the Hoodie client with synthetic workload, and performs basic operations.

*/ public class HoodieClientExample { - @Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table") - private String tablePath = "file:///tmp/hoodie/sample-table"; - - @Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table") - private String tableName = "hoodie_rt"; - - @Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ") - private String tableType = HoodieTableType.COPY_ON_WRITE.name(); - + private static Logger logger = LogManager.getLogger(HoodieClientExample.class); @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; - - private static Logger logger = LogManager.getLogger(HoodieClientExample.class); + @Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table") + private String tablePath = "file:///tmp/hoodie/sample-table"; + @Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table") + private String tableName = "hoodie_rt"; + @Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ") + private String tableType = HoodieTableType.COPY_ON_WRITE.name(); public static void main(String[] args) throws Exception { HoodieClientExample cli = new HoodieClientExample(); @@ -92,10 +86,10 @@ public class HoodieClientExample { // Create the write client to write some records in HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .forTable(tableName).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).build()) - .build(); + .forTable(tableName) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build()) + .withCompactionConfig( + HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); /** 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 c5544760d..881d0b7a8 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -118,13 +118,12 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { } private HoodieWriteConfig.Builder getConfigBuilder() { - return HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) - .forTable("test-trip-table").withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); + .forTable("test-trip-table") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); } private void assertNoWriteErrors(List statuses) { @@ -136,10 +135,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { private void assertPartitionMetadata(String[] partitionPaths, FileSystem fs) throws IOException { for (String partitionPath : partitionPaths) { - assertTrue( - HoodiePartitionMetadata.hasPartitionMetadata(fs, new Path(basePath, partitionPath))); - HoodiePartitionMetadata pmeta = new HoodiePartitionMetadata(fs, - new Path(basePath, partitionPath)); + assertTrue(HoodiePartitionMetadata.hasPartitionMetadata(fs, new Path(basePath, partitionPath))); + HoodiePartitionMetadata pmeta = new HoodiePartitionMetadata(fs, new Path(basePath, partitionPath)); pmeta.readFromFS(); assertEquals(3, pmeta.getPartitionDepth()); } @@ -148,8 +145,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { private void checkTaggedRecords(List taggedRecords, String commitTime) { for (HoodieRecord rec : taggedRecords) { assertTrue("Record " + rec + " found with no location.", rec.isCurrentLocationKnown()); - assertEquals( - "All records should have commit time " + commitTime + ", since updates were made", + assertEquals("All records should have commit time " + commitTime + ", since updates were made", rec.getCurrentLocation().getCommitTime(), commitTime); } } @@ -194,8 +190,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { List records = dataGen.generateInserts(newCommitTime, 200); JavaRDD writeRecords = jsc.parallelize(records, 1); - JavaRDD result = client.bulkInsertPreppedRecords(writeRecords, newCommitTime, - Option.empty()); + JavaRDD result = client.bulkInsertPreppedRecords(writeRecords, newCommitTime, Option.empty()); assertFalse("If Autocommit is false, then commit should not be made automatically", HoodieTestUtils.doesCommitExist(basePath, newCommitTime)); @@ -270,22 +265,16 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) - .getCommitTimeline(); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); - assertEquals("Latest commit should be 001", newCommitTime, - timeline.lastInstant().get().getTimestamp()); - assertEquals("Must contain 200 records", - records.size(), + assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp()); + assertEquals("Must contain 200 records", records.size(), HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); // Should have 100 records in table (check using Index), all in locations marked at commit HoodieReadClient readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); - List taggedRecords = readClient.tagLocation(jsc.parallelize(records, 1)) - .collect(); + List taggedRecords = readClient.tagLocation(jsc.parallelize(records, 1)).collect(); checkTaggedRecords(taggedRecords, "001"); /** @@ -309,10 +298,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // verify there are now 2 commits timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting two commits.", - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); - assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), - newCommitTime); + assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); + assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); // Index should be able to locate all updates in correct locations. readClient = new HoodieReadClient(jsc, hoodieWriteConfig.getBasePath()); @@ -324,8 +311,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { for (int i = 0; i < fullPartitionPaths.length; i++) { fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); } - assertEquals("Must contain 200 records", - 200, + assertEquals("Must contain 200 records", 200, HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); // Check that the incremental consumption from time 000 @@ -339,10 +325,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { @Test public void testUpsertsWithFinalizeWrite() throws Exception { - HoodieWriteConfig hoodieWriteConfig = getConfigBuilder() - .withUseTempFolderCopyOnWriteForCreate(true) - .withUseTempFolderCopyOnWriteForMerge(true) - .build(); + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder().withUseTempFolderCopyOnWriteForCreate(true) + .withUseTempFolderCopyOnWriteForMerge(true).build(); testUpsertsInternal(hoodieWriteConfig); } @@ -372,22 +356,16 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { assertNoWriteErrors(statuses); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) - .getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); - assertEquals("Latest commit should be 001", newCommitTime, - timeline.lastInstant().get().getTimestamp()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp()); assertEquals("Must contain 200 records", fewRecordsForInsert.size(), HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); // 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()); - List taggedRecords = index - .tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect(); + List taggedRecords = index.tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect(); checkTaggedRecords(taggedRecords, "001"); /** @@ -408,10 +386,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // verify there are now 2 commits timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - assertEquals("Expecting two commits.", - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); - assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), - newCommitTime); + assertEquals("Expecting two commits.", timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); + assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime); // Check the entire dataset has 150 records(200-50) still String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; @@ -422,14 +398,11 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { HoodieClientTestUtils.read(basePath, sqlContext, fs, fullPartitionPaths).count()); // Check that the incremental consumption from time 000 - assertEquals("Incremental consumption from latest commit, should give 50 updated records", - 50, + assertEquals("Incremental consumption from latest commit, should give 50 updated records", 50, HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); - assertEquals("Incremental consumption from time 001, should give 50 updated records", - 50, + assertEquals("Incremental consumption from time 001, should give 50 updated records", 50, HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "001").count()); - assertEquals("Incremental consumption from time 000, should give 150", - 150, + assertEquals("Incremental consumption from time 000, should give 150", 150, HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count()); } @@ -437,12 +410,10 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { @Test public void testCreateSavepoint() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) + HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) .build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieTestDataGenerator - .writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); + HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); /** * Write 1 (only inserts) @@ -451,8 +422,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 200); - List statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime) - .collect(); + List statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(statuses); /** @@ -496,12 +466,10 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // Verify there are no errors assertNoWriteErrors(statuses); - List partitionPaths = FSUtils - .getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); - HoodieTable table = HoodieTable - .getHoodieTable(metaClient, getConfig()); + List partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), + getConfig().shouldAssumeDatePartitioning()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); final TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView(); List dataFiles = partitionPaths.stream().flatMap(s -> { return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); @@ -511,8 +479,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // Delete savepoint assertFalse(table.getCompletedSavepointTimeline().empty()); - client.deleteSavepoint( - table.getCompletedSavepointTimeline().getInstants().findFirst().get().getTimestamp()); + client.deleteSavepoint(table.getCompletedSavepointTimeline().getInstants().findFirst().get().getTimestamp()); // rollback and reupsert 004 client.rollback(newCommitTime); @@ -535,12 +502,10 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { @Test public void testRollbackToSavepoint() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) + HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) .build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieTestDataGenerator - .writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); + HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); /** * Write 1 (only inserts) @@ -577,10 +542,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); // Verify there are no errors assertNoWriteErrors(statuses); - List partitionPaths = FSUtils - .getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + List partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), + getConfig().shouldAssumeDatePartitioning()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); @@ -618,8 +582,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { } // rollback to savepoint 002 - HoodieInstant savepoint = - table.getCompletedSavepointTimeline().getInstants().findFirst().get(); + HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get(); client.rollbackToSavepoint(savepoint.getTimestamp()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); @@ -646,8 +609,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { public void testInsertAndCleanByVersions() throws Exception { int maxVersions = 2; // keep upto 2 versions for each file HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) + HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) .retainFileVersions(maxVersions).build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); @@ -666,26 +628,22 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { assertNoWriteErrors(statuses); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) - .getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + 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()); assertFalse(table.getCompletedCommitTimeline().empty()); - String commitTime = - table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); + String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); assertFalse(table.getCompletedCleanTimeline().empty()); 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(); + List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect(); checkTaggedRecords(taggedRecords, newCommitTime); - // Keep doing some writes and clean inline. Make sure we have expected number of files remaining. + // Keep doing some writes and clean inline. Make sure we have expected number of files + // remaining. for (int writeCnt = 2; writeCnt < 10; writeCnt++) { Thread.sleep(1100); // make sure commits are unique @@ -696,8 +654,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // Verify there are no errors assertNoWriteErrors(statuses); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metadata, getConfig()); timeline = table.getCommitsTimeline(); @@ -707,20 +664,17 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // compute all the versions of all files, from time 0 HashMap> fileIdToVersions = new HashMap<>(); for (HoodieInstant entry : timeline.getInstants().collect(Collectors.toList())) { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(timeline.getInstantDetails(entry).get()); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(entry).get()); for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) { if (!fileIdToVersions.containsKey(wstat.getFileId())) { fileIdToVersions.put(wstat.getFileId(), new TreeSet<>()); } - fileIdToVersions.get(wstat.getFileId()) - .add(FSUtils.getCommitTime(new Path(wstat.getPath()).getName())); + fileIdToVersions.get(wstat.getFileId()).add(FSUtils.getCommitTime(new Path(wstat.getPath()).getName())); } } - List fileGroups = fsView.getAllFileGroups(partitionPath) - .collect(Collectors.toList()); + List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { // No file has no more than max versions @@ -733,10 +687,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // Each file, has the latest N versions (i.e cleaning gets rid of older versions) List commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId)); for (int i = 0; i < dataFiles.size(); i++) { - assertEquals( - "File " + fileId + " does not have latest versions on commits" + commitedVersions, - Iterables.get(dataFiles, i).getCommitTime(), - commitedVersions.get(commitedVersions.size() - 1 - i)); + assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions, + Iterables.get(dataFiles, i).getCommitTime(), commitedVersions.get(commitedVersions.size() - 1 - i)); } } } @@ -747,8 +699,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { public void testInsertAndCleanByCommits() throws Exception { int maxCommits = 3; // keep upto 3 commits from the past HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) + HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) .retainCommits(maxCommits).build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); @@ -767,28 +718,23 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { assertNoWriteErrors(statuses); // verify that there is a commit - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) - .getCommitTimeline(); - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + 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()); assertFalse(table.getCompletedCommitTimeline().empty()); - String commitTime = - table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); + String commitTime = table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp(); assertFalse(table.getCompletedCleanTimeline().empty()); 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(); + List taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect(); checkTaggedRecords(taggedRecords, newCommitTime); - // Keep doing some writes and clean inline. Make sure we have expected number of files remaining. + // Keep doing some writes and clean inline. Make sure we have expected number of files + // remaining. for (int writeCnt = 2; writeCnt < 10; writeCnt++) { Thread.sleep(1100); // make sure commits are unique newCommitTime = client.startCommit(); @@ -798,18 +744,14 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // Verify there are no errors assertNoWriteErrors(statuses); - HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg); HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline(); - Optional - earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); - Set acceptableCommits = - activeTimeline.getInstants().collect(Collectors.toSet()); + Optional earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); + Set acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet()); if (earliestRetainedCommit.isPresent()) { acceptableCommits.removeAll( - activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get().getTimestamp()) - .getInstants() + activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get().getTimestamp()).getInstants() .collect(Collectors.toSet())); acceptableCommits.add(earliestRetainedCommit.get()); } @@ -817,8 +759,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { TableFileSystemView fsView = table1.getFileSystemView(); // Need to ensure the following for (String partitionPath : dataGen.getPartitionPaths()) { - List fileGroups = fsView.getAllFileGroups(partitionPath) - .collect(Collectors.toList()); + List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { Set commitTimes = new HashSet<>(); fileGroup.getAllDataFiles().forEach(value -> { @@ -826,8 +767,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { commitTimes.add(value.getCommitTime()); }); assertEquals("Only contain acceptable versions of file should be present", - acceptableCommits.stream().map(HoodieInstant::getTimestamp) - .collect(Collectors.toSet()), commitTimes); + acceptableCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), commitTimes); } } } @@ -841,8 +781,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { String commitTime3 = "20160506030611"; new File(basePath + "/.hoodie").mkdirs(); HoodieTestDataGenerator - .writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, - basePath); + .writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); // Only first two have commit files HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2); @@ -864,10 +803,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32"); String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33"); - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY) - .build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, config, false); @@ -882,9 +819,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // Rollback commit3 client.rollback(commitTime3); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); // simulate partial failure, where .inflight was not deleted, but data files were. HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3); @@ -895,9 +832,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { client.rollback(commitTime2); assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); // simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a // .inflight commit and a bunch of data files around. @@ -909,20 +846,19 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { client.rollback(commitTime2); assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); // Let's rollback commit1, Check results client.rollback(commitTime1); assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime1)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); } - @Test public void testAutoRollbackCommit() throws Exception { // Let's create some commit files and parquet files @@ -931,8 +867,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { String commitTime3 = "20160506030611"; new File(basePath + "/.hoodie").mkdirs(); HoodieTestDataGenerator - .writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, - basePath); + .writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); // One good commit HoodieTestUtils.createCommitFiles(basePath, commitTime1); @@ -955,10 +890,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33"); // Turn auto rollback off - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY) - .build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); new HoodieWriteClient(jsc, config, false); @@ -966,62 +899,58 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); // Turn auto rollback on new HoodieWriteClient(jsc, config, true); assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2)); assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); - assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) || - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); - assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) && - HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33)); + assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22) + || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23)); + assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12) + && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13)); } private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) { HoodieWriteConfig.Builder builder = getConfigBuilder(); return builder.withCompactionConfig( - HoodieCompactionConfig.newBuilder() - .compactionSmallFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 15) + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 15) .insertSplitSize(insertSplitSize).build()) // tolerate upto 15 records - .withStorageConfig(HoodieStorageConfig.newBuilder() - .limitFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 20) - .build()) + .withStorageConfig( + HoodieStorageConfig.newBuilder().limitFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 20).build()) .build(); } @Test public void testSmallInsertHandlingForUpserts() throws Exception { - final String TEST_PARTITION_PATH = "2016/09/26"; - final int INSERT_SPLIT_LIMIT = 100; + final String testPartitionPath = "2016/09/26"; + final int insertSplitLimit = 100; // setup the small file handling params - HoodieWriteConfig config = getSmallInsertWriteConfig( - INSERT_SPLIT_LIMIT); // hold upto 200 records max - dataGen = new HoodieTestDataGenerator(new String[]{TEST_PARTITION_PATH}); + HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max + dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); HoodieWriteClient client = new HoodieWriteClient(jsc, config); // Inserts => will write file1 String commitTime1 = "001"; client.startCommitWithTime(commitTime1); - List inserts1 = dataGen - .generateInserts(commitTime1, INSERT_SPLIT_LIMIT); // this writes ~500kb + List inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb Set keys1 = HoodieClientTestUtils.getRecordKeys(inserts1); JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 1); @@ -1031,10 +960,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { assertEquals("Just 1 file needs to be added.", 1, statuses.size()); String file1 = statuses.get(0).getFileId(); - assertEquals("file should contain 100 records", - ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), new Path(basePath, - TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), - 100); + assertEquals("file should contain 100 records", ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), + new Path(basePath, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), 100); // Update + Inserts such that they just expand file1 String commitTime2 = "002"; @@ -1051,20 +978,16 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { assertEquals("Just 1 file needs to be updated.", 1, statuses.size()); assertEquals("Existing file should be expanded", file1, statuses.get(0).getFileId()); - assertEquals("Existing file should be expanded", commitTime1, - statuses.get(0).getStat().getPrevCommit()); - Path newFile = new Path(basePath, - TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); + assertEquals("Existing file should be expanded", commitTime1, statuses.get(0).getStat().getPrevCommit()); + Path newFile = new Path(basePath, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); assertEquals("file should contain 140 records", ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140); List records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile); for (GenericRecord record : records) { String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); - assertEquals("only expect commit2", commitTime2, - record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); - assertTrue("key expected to be part of commit2", - keys2.contains(recordKey) || keys1.contains(recordKey)); + assertEquals("only expect commit2", commitTime2, record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); + assertTrue("key expected to be part of commit2", keys2.contains(recordKey) || keys1.contains(recordKey)); } // update + inserts such that file1 is updated and expanded, a new file2 is created. @@ -1083,9 +1006,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metadata, config); TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView(); - List files = fileSystemView - .getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3).collect( - Collectors.toList()); + List files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) + .collect(Collectors.toList()); int numTotalInsertsInCommit3 = 0; for (HoodieDataFile file : files) { if (file.getFileName().contains(file1)) { @@ -1103,8 +1025,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { } } } - assertEquals("All keys added in commit 2 must be updated in commit3 correctly", 0, - keys2.size()); + assertEquals("All keys added in commit 2 must be updated in commit3 correctly", 0, keys2.size()); } else { assertEquals("New file must be written for commit 3", commitTime3, file.getCommitTime()); records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath())); @@ -1123,32 +1044,28 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { @Test public void testSmallInsertHandlingForInserts() throws Exception { - final String TEST_PARTITION_PATH = "2016/09/26"; - final int INSERT_SPLIT_LIMIT = 100; + final String testPartitionPath = "2016/09/26"; + final int insertSplitLimit = 100; // setup the small file handling params - HoodieWriteConfig config = getSmallInsertWriteConfig( - INSERT_SPLIT_LIMIT); // hold upto 200 records max - dataGen = new HoodieTestDataGenerator(new String[]{TEST_PARTITION_PATH}); + HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max + dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); HoodieWriteClient client = new HoodieWriteClient(jsc, config); // Inserts => will write file1 String commitTime1 = "001"; client.startCommitWithTime(commitTime1); - List inserts1 = dataGen - .generateInserts(commitTime1, INSERT_SPLIT_LIMIT); // this writes ~500kb + List inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb Set keys1 = HoodieClientTestUtils.getRecordKeys(inserts1); JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 1); List statuses = client.insert(insertRecordsRDD1, commitTime1).collect(); assertNoWriteErrors(statuses); - assertPartitionMetadata(new String[]{TEST_PARTITION_PATH}, fs); + assertPartitionMetadata(new String[] {testPartitionPath}, fs); assertEquals("Just 1 file needs to be added.", 1, statuses.size()); String file1 = statuses.get(0).getFileId(); - assertEquals("file should contain 100 records", - ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), new Path(basePath, - TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), - 100); + assertEquals("file should contain 100 records", ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), + new Path(basePath, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), 100); // Second, set of Inserts should just expand file1 String commitTime2 = "002"; @@ -1161,10 +1078,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { assertEquals("Just 1 file needs to be updated.", 1, statuses.size()); assertEquals("Existing file should be expanded", file1, statuses.get(0).getFileId()); - assertEquals("Existing file should be expanded", commitTime1, - statuses.get(0).getStat().getPrevCommit()); - Path newFile = new Path(basePath, - TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); + assertEquals("Existing file should be expanded", commitTime1, statuses.get(0).getStat().getPrevCommit()); + Path newFile = new Path(basePath, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); assertEquals("file should contain 140 records", ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140); @@ -1187,12 +1102,11 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { assertNoWriteErrors(statuses); assertEquals("2 files needs to be committed.", 2, statuses.size()); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); - List files = - table.getROFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3) - .collect(Collectors.toList()); + List files = table.getROFileSystemView() + .getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3) + .collect(Collectors.toList()); assertEquals("Total of 2 valid data files", 2, files.size()); int totalInserts = 0; @@ -1207,21 +1121,18 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { @Test public void testKeepLatestFileVersions() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) - .retainFileVersions(1).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) + .build(); // make 1 commit, with 1 file per partition HoodieTestUtils.createCommitFiles(basePath, "000"); String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000"); - HoodieTable table = HoodieTable - .getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List hoodieCleanStatsOne = table.clean(jsc); assertEquals("Must not clean any files", 0, @@ -1233,15 +1144,10 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // 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); - String file2P0C1 = HoodieTestUtils - .createNewDataFile(basePath, partitionPaths[0], "001"); // insert - String file2P1C1 = HoodieTestUtils - .createNewDataFile(basePath, partitionPaths[1], "001"); // insert + String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "001"); // insert + String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update @@ -1257,10 +1163,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // 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); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update @@ -1284,14 +1188,13 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { @Test public void testKeepLatestFileVersionsMOR() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) - .retainFileVersions(1).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) + .build(); - HoodieTableMetaClient metaClient = HoodieTestUtils - .initTableType(jsc.hadoopConfiguration(), basePath, HoodieTableType.MERGE_ON_READ); + HoodieTableMetaClient metaClient = HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, + HoodieTableType.MERGE_ON_READ); // Make 3 files, one base file and 2 log files associated with base file String file1P0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); @@ -1304,12 +1207,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // Make 4 files, one base file and 3 log files associated with base file HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0); - file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.empty()); - file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(2)); - file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(3)); + file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.empty()); + file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(2)); + file2P0L0 = HoodieTestUtils.createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(3)); // make 1 compaction commit HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); @@ -1318,19 +1218,15 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { assertEquals("Must clean three files, one parquet and 2 log files", 3, getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size()); assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0)); - assertFalse(HoodieTestUtils - .doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.empty())); - assertFalse(HoodieTestUtils - .doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.of(2))); + assertFalse(HoodieTestUtils.doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.empty())); + assertFalse(HoodieTestUtils.doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.of(2))); } @Test public void testKeepLatestCommits() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) - .retainCommits(2).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); // make 1 commit, with 1 file per partition HoodieTestUtils.createCommitFiles(basePath, "000"); @@ -1351,13 +1247,11 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // 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); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); - String file2P0C1 = HoodieTestUtils - .createNewDataFile(basePath, partitionPaths[0], "001"); // insert - String file2P1C1 = HoodieTestUtils - .createNewDataFile(basePath, partitionPaths[1], "001"); // insert + String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "001"); // insert + String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update @@ -1373,36 +1267,30 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // 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); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002"); List hoodieCleanStatsThree = table.clean(jsc); - assertEquals( - "Must not clean any file. We have to keep 1 version before the latest commit time to keep", - 0, getCleanStat(hoodieCleanStatsThree, partitionPaths[0]).getSuccessDeleteFiles().size()); + assertEquals("Must not clean any file. We have to keep 1 version before the latest commit time to keep", 0, + getCleanStat(hoodieCleanStatsThree, partitionPaths[0]).getSuccessDeleteFiles().size()); assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0)); // 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); + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file2P0C1); // update String file4P0C3 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "003"); List hoodieCleanStatsFour = table.clean(jsc); - assertEquals( - "Must not clean one old file", 1, + assertEquals("Must not clean one old file", 1, getCleanStat(hoodieCleanStatsFour, partitionPaths[0]).getSuccessDeleteFiles().size()); assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0)); @@ -1424,34 +1312,27 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { @Test public void testCleaningWithZeroPartitonPaths() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) - .retainCommits(2).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); // Make a commit, although there are no partitionPaths. // Example use-case of this is when a client wants to create a table // with just some commit metadata, but no data/partitionPaths. HoodieTestUtils.createCommitFiles(basePath, "000"); - HoodieTable table = HoodieTable - .getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List hoodieCleanStatsOne = table.clean(jsc); - assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", - hoodieCleanStatsOne.isEmpty()); + assertTrue("HoodieCleanStats should be empty for a table with empty partitionPaths", hoodieCleanStatsOne.isEmpty()); } @Test public void testCleaningSkewedPartitons() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) - .retainCommits(2).build()).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy( + HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build(); Map stageOneShuffleReadTaskRecordsCountMap = new HashMap<>(); // Since clean involves repartition in order to uniformly distribute data, @@ -1465,16 +1346,12 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { @Override public void onTaskEnd(SparkListenerTaskEnd taskEnd) { - Iterator> iterator = taskEnd.taskMetrics().accumulators() - .iterator(); + Iterator> iterator = taskEnd.taskMetrics().accumulators().iterator(); while (iterator.hasNext()) { AccumulatorV2 accumulator = iterator.next(); - if (taskEnd.stageId() == 1 && - accumulator.isRegistered() && - accumulator.name().isDefined() && - accumulator.name().get().equals("internal.metrics.shuffle.read.recordsRead")) { - stageOneShuffleReadTaskRecordsCountMap - .put(taskEnd.taskInfo().taskId(), (Long) accumulator.value()); + if (taskEnd.stageId() == 1 && accumulator.isRegistered() && accumulator.name().isDefined() + && accumulator.name().get().equals("internal.metrics.shuffle.read.recordsRead")) { + stageOneShuffleReadTaskRecordsCountMap.put(taskEnd.taskInfo().taskId(), (Long) accumulator.value()); } } } @@ -1501,63 +1378,53 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { updateAllFilesInPartition(filesP1C0, partitionPaths[1], "003"); updateAllFilesInPartition(filesP2C0, partitionPaths[2], "003"); - HoodieTable table = HoodieTable - .getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), - config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); List hoodieCleanStats = table.clean(jsc); - assertEquals(100, - getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size()); - assertEquals(10, - getCleanStat(hoodieCleanStats, partitionPaths[1]).getSuccessDeleteFiles().size()); - assertEquals(10, - getCleanStat(hoodieCleanStats, partitionPaths[2]).getSuccessDeleteFiles().size()); + assertEquals(100, getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size()); + assertEquals(10, getCleanStat(hoodieCleanStats, partitionPaths[1]).getSuccessDeleteFiles().size()); + assertEquals(10, getCleanStat(hoodieCleanStats, partitionPaths[2]).getSuccessDeleteFiles().size()); // 3 tasks are expected since the number of partitions is 3 assertEquals(3, stageOneShuffleReadTaskRecordsCountMap.keySet().size()); // Sum of all records processed = total number of files to clean - assertEquals(120, stageOneShuffleReadTaskRecordsCountMap - .values().stream().reduce((a, b) -> a + b).get().intValue()); + assertEquals(120, + stageOneShuffleReadTaskRecordsCountMap.values().stream().reduce((a, b) -> a + b).get().intValue()); assertTrue("The skew in handling files to clean is not removed. " + "Each task should handle more records than the partitionPath with least files " + "and less records than the partitionPath with most files.", - stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100) - .count() == 3); + stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100).count() == 3); } @Test public void testCleanTemporaryDataFiles() throws IOException { HoodieTestUtils.createCommitFiles(basePath, "000"); List tempFiles = createTempFiles("000", 10); - assertEquals("Some temp files are created.",10, tempFiles.size()); - assertEquals("Some temp files are created.",tempFiles.size(), getTotalTempFiles()); + assertEquals("Some temp files are created.", 10, tempFiles.size()); + assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withUseTempFolderCopyOnWriteForCreate(false) .withUseTempFolderCopyOnWriteForMerge(false).build(); - HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), - config.getBasePath(), true), - config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); table.rollback(jsc, Collections.emptyList()); - assertEquals("Some temp files are created.",tempFiles.size(), getTotalTempFiles()); + assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles()); - config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withUseTempFolderCopyOnWriteForCreate(true) + config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true) .withUseTempFolderCopyOnWriteForMerge(false).build(); - table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), - config.getBasePath(), true), + table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config); table.rollback(jsc, Collections.emptyList()); - assertEquals("All temp files are deleted.",0, getTotalTempFiles()); + assertEquals("All temp files are deleted.", 0, getTotalTempFiles()); } public void testCommitWritesRelativePaths() throws Exception { HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg); String commitTime = "000"; @@ -1574,11 +1441,10 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { // Get parquet file paths from commit metadata String actionType = table.getCommitActionType(); - HoodieInstant commitInstant = - new HoodieInstant(false, actionType, commitTime); + HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime); HoodieTimeline commitTimeline = table.getCommitTimeline().filterCompletedInstants(); - HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commitInstant).get()); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(commitTimeline.getInstantDetails(commitInstant).get()); String basePath = table.getMetaClient().getBasePath(); Collection commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values(); @@ -1596,15 +1462,12 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable { } } - private HoodieCleanStat getCleanStat(List hoodieCleanStatsTwo, - String partitionPath) { - return hoodieCleanStatsTwo.stream() - .filter(e -> e.getPartitionPath().equals(partitionPath)) - .findFirst().get(); + private HoodieCleanStat getCleanStat(List hoodieCleanStatsTwo, String partitionPath) { + return hoodieCleanStatsTwo.stream().filter(e -> e.getPartitionPath().equals(partitionPath)).findFirst().get(); } - private void updateAllFilesInPartition(List files, String partitionPath, - String commitTime) throws IOException { + private void updateAllFilesInPartition(List files, String partitionPath, String commitTime) + throws IOException { for (String fileId : files) { HoodieTestUtils.createDataFile(basePath, partitionPath, commitTime, fileId); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java index 0dc7a816f..c9969f2a6 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java @@ -16,7 +16,6 @@ package com.uber.hoodie; - import static org.junit.Assert.assertEquals; import com.uber.hoodie.common.HoodieClientTestUtils; @@ -58,11 +57,11 @@ public class TestMultiFS implements Serializable { private static MiniDFSCluster dfsCluster; private static DistributedFileSystem dfs; private static Logger logger = LogManager.getLogger(TestMultiFS.class); + private static JavaSparkContext jsc; + private static SQLContext sqlContext; private String tablePath = "file:///tmp/hoodie/sample-table"; private String tableName = "hoodie_rt"; private String tableType = HoodieTableType.COPY_ON_WRITE.name(); - private static JavaSparkContext jsc; - private static SQLContext sqlContext; @BeforeClass public static void initClass() throws Exception { @@ -92,7 +91,8 @@ public class TestMultiFS implements Serializable { hdfsTestService.stop(); dfsCluster.shutdown(); } - // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the + // same JVM FileSystem.closeAll(); } @@ -111,8 +111,7 @@ public class TestMultiFS implements Serializable { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(dfsBasePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable(tableName).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc, cfg); // Write generated data to hdfs (only inserts) @@ -125,10 +124,8 @@ public class TestMultiFS implements Serializable { // Read from hdfs FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient) - .getCommitTimeline(); - Dataset readRecords = HoodieClientTestUtils - .readCommit(dfsBasePath, sqlContext, timeline, readCommitTime); + HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + Dataset readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime); assertEquals("Should contain 100 records", readRecords.count(), records.size()); // Write to local @@ -138,8 +135,7 @@ public class TestMultiFS implements Serializable { HoodieWriteConfig localConfig = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable(tableName).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); HoodieWriteClient localWriteClient = new HoodieWriteClient(jsc, localConfig); String writeCommitTime = localWriteClient.startCommit(); @@ -153,8 +149,7 @@ public class TestMultiFS implements Serializable { fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf()); metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath); timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); - Dataset localReadRecords = HoodieClientTestUtils - .readCommit(tablePath, sqlContext, timeline, writeCommitTime); + Dataset localReadRecords = HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime); assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size()); } } 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 295d41525..b87e1768c 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 @@ -69,8 +69,7 @@ public class HoodieClientTestUtils { return keys; } - private static void fakeMetaFile(String basePath, String commitTime, String suffix) - throws IOException { + private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException { String parentPath = basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME; new File(parentPath).mkdirs(); new File(parentPath + "/" + commitTime + suffix).createNewFile(); @@ -85,55 +84,48 @@ public class HoodieClientTestUtils { fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION); } - public static void fakeDataFile(String basePath, String partitionPath, String commitTime, - String fileId) throws Exception { + public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId) + throws Exception { fakeDataFile(basePath, partitionPath, commitTime, fileId, 0); } - public static void fakeDataFile(String basePath, String partitionPath, String commitTime, - String fileId, long length) throws Exception { + public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId, long length) + throws Exception { String parentPath = String.format("%s/%s", basePath, partitionPath); new File(parentPath).mkdirs(); - String path = String - .format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId)); + String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId)); new File(path).createNewFile(); new RandomAccessFile(path, "rw").setLength(length); } public static SparkConf getSparkConfForTest(String appName) { - SparkConf sparkConf = new SparkConf() - .setAppName(appName) + System.out.println("HIII" + "HII2"); + SparkConf sparkConf = new SparkConf().setAppName(appName) .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .setMaster("local[1]"); return HoodieReadClient.addHoodieSupport(sparkConf); } - public static HashMap getLatestFileIDsToFullPath(String basePath, - HoodieTimeline commitTimeline, + public static HashMap getLatestFileIDsToFullPath(String basePath, HoodieTimeline commitTimeline, List commitsToReturn) throws IOException { HashMap fileIdToFullPath = new HashMap<>(); for (HoodieInstant commit : commitsToReturn) { - HoodieCommitMetadata metadata = - HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get()); + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get()); fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath)); } return fileIdToFullPath; } - public static Dataset readCommit(String basePath, - SQLContext sqlContext, - HoodieTimeline commitTimeline, + public static Dataset readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline, String commitTime) { - HoodieInstant commitInstant = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); + HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); if (!commitTimeline.containsInstant(commitInstant)) { new HoodieException("No commit exists at " + commitTime); } try { HashMap paths = getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant)); - return sqlContext.read() - .parquet(paths.values().toArray(new String[paths.size()])) + return sqlContext.read().parquet(paths.values().toArray(new String[paths.size()])) .filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime)); } catch (Exception e) { throw new HoodieException("Error reading commit " + commitTime, e); @@ -143,50 +135,37 @@ public class HoodieClientTestUtils { /** * Obtain all new data written into the Hoodie dataset since the given timestamp. */ - public static Dataset readSince(String basePath, - SQLContext sqlContext, - HoodieTimeline commitTimeline, + public static Dataset readSince(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline, String lastCommitTime) { - List commitsToReturn = - commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE) - .getInstants().collect(Collectors.toList()); + List commitsToReturn = commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE) + .getInstants().collect(Collectors.toList()); try { // Go over the commit metadata, and obtain the new files that need to be read. - HashMap fileIdToFullPath = getLatestFileIDsToFullPath(basePath, - commitTimeline, commitsToReturn); - return sqlContext.read() - .parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()])) - .filter( - String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime)); + HashMap fileIdToFullPath = getLatestFileIDsToFullPath(basePath, commitTimeline, commitsToReturn); + return sqlContext.read().parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()])) + .filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime)); } catch (IOException e) { - throw new HoodieException( - "Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e); + throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e); } } /** * 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(String basePath, SQLContext sqlContext, FileSystem fs, String... paths) { List filteredPaths = new ArrayList<>(); try { HoodieTable hoodieTable = HoodieTable .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null); for (String path : paths) { TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView( - hoodieTable.getMetaClient(), - hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path))); - List latestFiles = fileSystemView.getLatestDataFiles().collect( - Collectors.toList()); + hoodieTable.getMetaClient(), hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path))); + List latestFiles = fileSystemView.getLatestDataFiles().collect(Collectors.toList()); for (HoodieDataFile file : latestFiles) { filteredPaths.add(file.getPath()); } } - return sqlContext.read() - .parquet(filteredPaths.toArray(new String[filteredPaths.size()])); + return sqlContext.read().parquet(filteredPaths.toArray(new String[filteredPaths.size()])); } catch (Exception e) { throw new HoodieException("Error reading hoodie dataset as a dataframe", e); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java index c7e3e9156..d925c5669 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java @@ -42,8 +42,7 @@ import org.apache.hadoop.mapred.RecordReader; */ public class HoodieMergeOnReadTestUtils { - public static List getRecordsUsingInputFormat(List inputPaths, - String basePath) + public static List getRecordsUsingInputFormat(List inputPaths, String basePath) throws IOException { JobConf jobConf = new JobConf(); Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA)); @@ -59,7 +58,8 @@ public class HoodieMergeOnReadTestUtils { ArrayWritable writable = (ArrayWritable) recordReader.createValue(); while (recordReader.next(key, writable)) { GenericRecordBuilder newRecord = new GenericRecordBuilder(schema); - // writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno] + // writable returns an array with [field1, field2, _hoodie_commit_time, + // _hoodie_commit_seqno] Writable[] values = writable.get(); schema.getFields().forEach(field -> { newRecord.set(field, values[2]); @@ -76,12 +76,11 @@ public class HoodieMergeOnReadTestUtils { }).get(); } - private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, - Schema schema, String basePath) { + private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, Schema schema, + String basePath) { List fields = schema.getFields(); String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); - String postions = fields.stream().map(f -> String.valueOf(f.pos())) - .collect(Collectors.joining(",")); + String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(",")); Configuration conf = HoodieTestUtils.getDefaultHadoopConf(); jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names); jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions); 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 beb76df65..f623fc7cb 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 @@ -41,20 +41,15 @@ import org.apache.hadoop.fs.Path; /** * Class to be used in tests to keep generating test inserts and updates against a corpus. - * + *

* Test data uses a toy Uber trips, data model. */ public class HoodieTestDataGenerator { - static class KeyPartition { - - HoodieKey key; - String partitionPath; - } - - public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\"," - + "\"name\": \"triprec\"," - + "\"fields\": [ " + // based on examination of sample file, the schema produces the following per record size + public static final int SIZE_PER_RECORD = 50 * 1024; + public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"}; + public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\"," + "\"name\": \"triprec\"," + "\"fields\": [ " + "{\"name\": \"timestamp\",\"type\": \"double\"}," + "{\"name\": \"_row_key\", \"type\": \"string\"}," + "{\"name\": \"rider\", \"type\": \"string\"}," @@ -64,25 +59,9 @@ public class HoodieTestDataGenerator { + "{\"name\": \"end_lat\", \"type\": \"double\"}," + "{\"name\": \"end_lon\", \"type\": \"double\"}," + "{\"name\":\"fare\",\"type\": \"double\"}]}"; - - // based on examination of sample file, the schema produces the following per record size - public static final int SIZE_PER_RECORD = 50 * 1024; - - public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"}; - - - public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, - String basePath) { - for (String partitionPath : partitionPaths) { - new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath)) - .trySave(0); - } - } - - private List existingKeysList = new ArrayList<>(); - public static Schema avroSchema = HoodieAvroUtils - .addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA)); + public static Schema avroSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA)); private static Random rand = new Random(46474747); + private List existingKeysList = new ArrayList<>(); private String[] partitionPaths = DEFAULT_PARTITION_PATHS; public HoodieTestDataGenerator(String[] partitionPaths) { @@ -90,13 +69,69 @@ public class HoodieTestDataGenerator { } public HoodieTestDataGenerator() { - this(new String[]{"2016/03/15", "2015/03/16", "2015/03/17"}); + this(new String[] {"2016/03/15", "2015/03/16", "2015/03/17"}); } + public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, String basePath) { + for (String partitionPath : partitionPaths) { + new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath)).trySave(0); + } + } /** - * Generates new inserts, uniformly across the partition paths above. It also updates the list of - * existing keys. + * Generates a new avro record of the above schema format, retaining the key if optionally provided. + */ + public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) throws IOException { + GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0); + HoodieAvroUtils.addCommitMetadataToRecord(rec, commitTime, "-1"); + return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA); + } + + public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName, + double timestamp) { + GenericRecord rec = new GenericData.Record(avroSchema); + rec.put("_row_key", rowKey); + rec.put("timestamp", timestamp); + rec.put("rider", riderName); + rec.put("driver", driverName); + rec.put("begin_lat", rand.nextDouble()); + rec.put("begin_lon", rand.nextDouble()); + rec.put("end_lat", rand.nextDouble()); + rec.put("end_lon", rand.nextDouble()); + rec.put("fare", rand.nextDouble() * 100); + return rec; + } + + public static void createCommitFile(String basePath, String commitTime) throws IOException { + Path commitFile = new Path( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime)); + FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + FSDataOutputStream os = fs.create(commitFile, true); + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + try { + // Write empty commit metadata + os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } finally { + os.close(); + } + } + + public static void createSavepointFile(String basePath, String commitTime) throws IOException { + Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + + "/" + HoodieTimeline.makeSavePointFileName(commitTime)); + FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + FSDataOutputStream os = fs.create(commitFile, true); + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + try { + // Write empty commit metadata + os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } finally { + os.close(); + } + } + + /** + * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys. */ public List generateInserts(String commitTime, int n) throws IOException { List inserts = new ArrayList<>(); @@ -119,8 +154,7 @@ public class HoodieTestDataGenerator { return generateDeletesFromExistingRecords(inserts); } - public List generateDeletesFromExistingRecords(List existingRecords) - throws IOException { + public List generateDeletesFromExistingRecords(List existingRecords) throws IOException { List deletes = new ArrayList<>(); for (HoodieRecord existingRecord : existingRecords) { HoodieRecord record = generateDeleteRecord(existingRecord); @@ -132,17 +166,15 @@ public class HoodieTestDataGenerator { public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException { HoodieKey key = existingRecord.getKey(); - TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), - key.getPartitionPath(), null, true); + TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), key.getPartitionPath(), + null, true); return new HoodieRecord(key, payload); } - public List generateUpdates(String commitTime, List baseRecords) - throws IOException { + public List generateUpdates(String commitTime, List baseRecords) throws IOException { List updates = new ArrayList<>(); for (HoodieRecord baseRecord : baseRecords) { - HoodieRecord record = new HoodieRecord(baseRecord.getKey(), - generateRandomValue(baseRecord.getKey(), commitTime)); + HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateRandomValue(baseRecord.getKey(), commitTime)); updates.add(record); } return updates; @@ -161,68 +193,13 @@ public class HoodieTestDataGenerator { return updates; } - - /** - * Generates a new avro record of the above schema format, retaining the key if optionally - * provided. - */ - public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) - throws IOException { - GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, - "driver-" + commitTime, 0.0); - HoodieAvroUtils.addCommitMetadataToRecord(rec, commitTime, "-1"); - return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), - TRIP_EXAMPLE_SCHEMA); - } - - public static GenericRecord generateGenericRecord(String rowKey, String riderName, - String driverName, double timestamp) { - GenericRecord rec = new GenericData.Record(avroSchema); - rec.put("_row_key", rowKey); - rec.put("timestamp", timestamp); - rec.put("rider", riderName); - rec.put("driver", driverName); - rec.put("begin_lat", rand.nextDouble()); - rec.put("begin_lon", rand.nextDouble()); - rec.put("end_lat", rand.nextDouble()); - rec.put("end_lon", rand.nextDouble()); - rec.put("fare", rand.nextDouble() * 100); - return rec; - } - - public static void createCommitFile(String basePath, String commitTime) throws IOException { - Path commitFile = - new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCommitFileName(commitTime)); - FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); - FSDataOutputStream os = fs.create(commitFile, true); - HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); - try { - // Write empty commit metadata - os.writeBytes(new String(commitMetadata.toJsonString().getBytes( - StandardCharsets.UTF_8))); - } finally { - os.close(); - } - } - - public static void createSavepointFile(String basePath, String commitTime) throws IOException { - Path commitFile = - new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeSavePointFileName(commitTime)); - FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); - FSDataOutputStream os = fs.create(commitFile, true); - HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); - try { - // Write empty commit metadata - os.writeBytes(new String(commitMetadata.toJsonString().getBytes( - StandardCharsets.UTF_8))); - } finally { - os.close(); - } - } - public String[] getPartitionPaths() { return partitionPaths; } + + static class KeyPartition { + + HoodieKey key; + String partitionPath; + } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java b/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java index 11c790f66..b73cba791 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java @@ -43,15 +43,15 @@ import org.apache.commons.io.IOUtils; */ public class TestRawTripPayload implements HoodieRecordPayload { - private transient static final ObjectMapper mapper = new ObjectMapper(); + private static final transient ObjectMapper mapper = new ObjectMapper(); private String partitionPath; private String rowKey; private byte[] jsonDataCompressed; private int dataSize; private boolean isDeleted; - public TestRawTripPayload(Optional jsonData, String rowKey, String partitionPath, - String schemaStr, Boolean isDeleted) throws IOException { + public TestRawTripPayload(Optional jsonData, String rowKey, String partitionPath, String schemaStr, + Boolean isDeleted) throws IOException { if (jsonData.isPresent()) { this.jsonDataCompressed = compressData(jsonData.get()); this.dataSize = jsonData.get().length(); @@ -61,8 +61,7 @@ public class TestRawTripPayload implements HoodieRecordPayload combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) - throws IOException { + public Optional combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException { return this.getInsertValue(schema); } @@ -120,8 +118,7 @@ public class TestRawTripPayload implements HoodieRecordPayload mergedMetadataMap = new HashMap<>(); + public static Map mergeMetadataForWriteStatuses(List writeStatuses) { + Map allWriteStatusMergedMetadataMap = new HashMap<>(); + for (WriteStatus writeStatus : writeStatuses) { + MetadataMergeWriteStatus.mergeMetadataMaps(((MetadataMergeWriteStatus) writeStatus).getMergedMetadataMap(), + allWriteStatusMergedMetadataMap); + } + return allWriteStatusMergedMetadataMap; + } + + private static void mergeMetadataMaps(Map mergeFromMap, Map mergeToMap) { + for (Entry entry : mergeFromMap.entrySet()) { + String key = entry.getKey(); + if (!mergeToMap.containsKey(key)) { + mergeToMap.put(key, "0"); + } + mergeToMap.put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key))); + } + } + + private static String addStrsAsInt(String a, String b) { + return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b)); + } + @Override public void markSuccess(HoodieRecord record, Optional> recordMetadata) { super.markSuccess(record, recordMetadata); @@ -156,43 +176,15 @@ public class TestRawTripPayload implements HoodieRecordPayload> recordMetadata) { + public void markFailure(HoodieRecord record, Throwable t, Optional> recordMetadata) { super.markFailure(record, t, recordMetadata); if (recordMetadata.isPresent()) { mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap); } } - public static Map mergeMetadataForWriteStatuses( - List writeStatuses) { - Map allWriteStatusMergedMetadataMap = new HashMap<>(); - for (WriteStatus writeStatus : writeStatuses) { - MetadataMergeWriteStatus.mergeMetadataMaps( - ((MetadataMergeWriteStatus) writeStatus).getMergedMetadataMap(), - allWriteStatusMergedMetadataMap); - } - return allWriteStatusMergedMetadataMap; - } - - private static void mergeMetadataMaps(Map mergeFromMap, - Map mergeToMap) { - for (Entry entry : mergeFromMap.entrySet()) { - String key = entry.getKey(); - if (!mergeToMap.containsKey(key)) { - mergeToMap.put(key, "0"); - } - mergeToMap - .put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key))); - } - } - private Map getMergedMetadataMap() { return mergedMetadataMap; } - - private static String addStrsAsInt(String a, String b) { - return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b)); - } } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/config/HoodieWriteConfigTest.java b/hoodie-client/src/test/java/com/uber/hoodie/config/HoodieWriteConfigTest.java index 395197bf9..3e6fc8e68 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/config/HoodieWriteConfigTest.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/config/HoodieWriteConfigTest.java @@ -49,8 +49,7 @@ public class HoodieWriteConfigTest { assertEquals(config.getMinCommitsToKeep(), 2); } - private ByteArrayOutputStream saveParamsIntoOutputStream(Map params) - throws IOException { + private ByteArrayOutputStream saveParamsIntoOutputStream(Map params) throws IOException { Properties properties = new Properties(); properties.putAll(params); ByteArrayOutputStream outStream = new ByteArrayOutputStream(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java index 35e36ef1d..6c47f2d4c 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBufferedIterator.java @@ -16,18 +16,13 @@ package com.uber.hoodie.func; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.exception.HoodieException; -import org.apache.avro.generic.IndexedRecord; -import org.apache.commons.io.FileUtils; -import org.apache.spark.util.SizeEstimator; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - import java.io.IOException; import java.util.Iterator; import java.util.List; @@ -37,171 +32,168 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.Semaphore; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import org.apache.avro.generic.IndexedRecord; +import org.apache.commons.io.FileUtils; +import org.apache.spark.util.SizeEstimator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; public class TestBufferedIterator { - private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); - private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); - private ExecutorService recordReader = null; + private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); + private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); + private ExecutorService recordReader = null; - @Before - public void beforeTest() { - this.recordReader = Executors.newFixedThreadPool(1); + @Before + public void beforeTest() { + this.recordReader = Executors.newFixedThreadPool(1); + } + + @After + public void afterTest() { + if (this.recordReader != null) { + this.recordReader.shutdownNow(); + this.recordReader = null; + } + } + + // Test to ensure that we are reading all records from buffered iterator in the same order + // without any exceptions. + @Test(timeout = 60000) + public void testRecordReading() throws IOException, ExecutionException, InterruptedException { + final int numRecords = 128; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), FileUtils.ONE_KB, + HoodieTestDataGenerator.avroSchema); + Future result = recordReader.submit(() -> { + bufferedIterator.startBuffering(); + return true; + }); + final Iterator originalRecordIterator = hoodieRecords.iterator(); + int recordsRead = 0; + while (bufferedIterator.hasNext()) { + final HoodieRecord originalRecord = originalRecordIterator.next(); + final Optional originalInsertValue = originalRecord.getData() + .getInsertValue(HoodieTestDataGenerator.avroSchema); + final BufferedIterator.BufferedIteratorPayload payload = bufferedIterator.next(); + // Ensure that record ordering is guaranteed. + Assert.assertEquals(originalRecord, payload.record); + // cached insert value matches the expected insert value. + Assert.assertEquals(originalInsertValue, payload.insertValue); + recordsRead++; + } + Assert.assertFalse(bufferedIterator.hasNext() || originalRecordIterator.hasNext()); + // all the records should be read successfully. + Assert.assertEquals(numRecords, recordsRead); + // should not throw any exceptions. + Assert.assertTrue(result.get()); + } + + // Test to ensure that record buffering is throttled when we hit memory limit. + @Test(timeout = 60000) + public void testMemoryLimitForBuffering() throws IOException, InterruptedException { + final int numRecords = 128; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + // maximum number of records to keep in memory. + final int recordLimit = 5; + final long memoryLimitInBytes = recordLimit * SizeEstimator.estimate(hoodieRecords.get(0)); + final BufferedIterator bufferedIterator = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, + HoodieTestDataGenerator.avroSchema); + Future result = recordReader.submit(() -> { + bufferedIterator.startBuffering(); + return true; + }); + // waiting for permits to expire. + while (!isQueueFull(bufferedIterator.rateLimiter)) { + Thread.sleep(10); + } + Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); + Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); + Assert.assertEquals(recordLimit, bufferedIterator.size()); + Assert.assertEquals(recordLimit - 1, bufferedIterator.samplingRecordCounter.get()); + + // try to read 2 records. + Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().record); + Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().record); + + // waiting for permits to expire. + while (!isQueueFull(bufferedIterator.rateLimiter)) { + Thread.sleep(10); + } + // No change is expected in rate limit or number of buffered records. We only expect + // buffering thread to read + // 2 more records into the buffer. + Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); + Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); + Assert.assertEquals(recordLimit, bufferedIterator.size()); + Assert.assertEquals(recordLimit - 1 + 2, bufferedIterator.samplingRecordCounter.get()); + } + + // Test to ensure that exception in either buffering thread or BufferedIterator-reader thread + // is propagated to + // another thread. + @Test(timeout = 60000) + public void testException() throws IOException, InterruptedException { + final int numRecords = 256; + final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); + // buffer memory limit + final long memoryLimitInBytes = 4 * SizeEstimator.estimate(hoodieRecords.get(0)); + + // first let us throw exception from bufferIterator reader and test that buffering thread + // stops and throws + // correct exception back. + BufferedIterator bufferedIterator1 = new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, + HoodieTestDataGenerator.avroSchema); + Future result = recordReader.submit(() -> { + bufferedIterator1.startBuffering(); + return true; + }); + // waiting for permits to expire. + while (!isQueueFull(bufferedIterator1.rateLimiter)) { + Thread.sleep(10); + } + // notify buffering thread of an exception and ensure that it exits. + final Exception e = new Exception("Failing it :)"); + bufferedIterator1.markAsFailed(e); + try { + result.get(); + Assert.fail("exception is expected"); + } catch (ExecutionException e1) { + Assert.assertEquals(HoodieException.class, e1.getCause().getClass()); + Assert.assertEquals(e, e1.getCause().getCause()); } - @After - public void afterTest() { - if (this.recordReader != null) { - this.recordReader.shutdownNow(); - this.recordReader = null; - } + // second let us raise an exception while doing record buffering. this exception should get + // propagated to + // buffered iterator reader. + final RuntimeException expectedException = new RuntimeException("failing record reading"); + final Iterator mockHoodieRecordsIterator = mock(Iterator.class); + when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); + when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); + BufferedIterator bufferedIterator2 = new BufferedIterator(mockHoodieRecordsIterator, memoryLimitInBytes, + HoodieTestDataGenerator.avroSchema); + Future result2 = recordReader.submit(() -> { + bufferedIterator2.startBuffering(); + return true; + }); + try { + bufferedIterator2.hasNext(); + Assert.fail("exception is expected"); + } catch (Exception e1) { + Assert.assertEquals(expectedException, e1.getCause()); } - - // Test to ensure that we are reading all records from buffered iterator in the same order without any exceptions. - @Test(timeout = 60000) - public void testRecordReading() throws IOException, ExecutionException, InterruptedException { - final int numRecords = 128; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - final BufferedIterator bufferedIterator = - new BufferedIterator(hoodieRecords.iterator(), FileUtils.ONE_KB, HoodieTestDataGenerator.avroSchema); - Future result = - recordReader.submit( - () -> { - bufferedIterator.startBuffering(); - return true; - } - ); - final Iterator originalRecordIterator = hoodieRecords.iterator(); - int recordsRead = 0; - while (bufferedIterator.hasNext()) { - final HoodieRecord originalRecord = originalRecordIterator.next(); - final Optional originalInsertValue = - originalRecord.getData().getInsertValue(HoodieTestDataGenerator.avroSchema); - final BufferedIterator.BufferedIteratorPayload payload = bufferedIterator.next(); - // Ensure that record ordering is guaranteed. - Assert.assertEquals(originalRecord, payload.record); - // cached insert value matches the expected insert value. - Assert.assertEquals(originalInsertValue, payload.insertValue); - recordsRead++; - } - Assert.assertFalse(bufferedIterator.hasNext() || originalRecordIterator.hasNext()); - // all the records should be read successfully. - Assert.assertEquals(numRecords, recordsRead); - // should not throw any exceptions. - Assert.assertTrue(result.get()); + // buffering thread should also have exited. make sure that it is not running. + try { + result2.get(); + Assert.fail("exception is expected"); + } catch (ExecutionException e2) { + Assert.assertEquals(expectedException, e2.getCause()); } + } - // Test to ensure that record buffering is throttled when we hit memory limit. - @Test(timeout = 60000) - public void testMemoryLimitForBuffering() throws IOException, InterruptedException { - final int numRecords = 128; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - // maximum number of records to keep in memory. - final int recordLimit = 5; - final long memoryLimitInBytes = recordLimit * SizeEstimator.estimate(hoodieRecords.get(0)); - final BufferedIterator bufferedIterator = - new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, HoodieTestDataGenerator.avroSchema); - Future result = - recordReader.submit( - () -> { - bufferedIterator.startBuffering(); - return true; - } - ); - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator.rateLimiter)) { - Thread.sleep(10); - } - Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); - Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); - Assert.assertEquals(recordLimit, bufferedIterator.size()); - Assert.assertEquals(recordLimit - 1, bufferedIterator.samplingRecordCounter.get()); - - // try to read 2 records. - Assert.assertEquals(hoodieRecords.get(0), bufferedIterator.next().record); - Assert.assertEquals(hoodieRecords.get(1), bufferedIterator.next().record); - - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator.rateLimiter)) { - Thread.sleep(10); - } - // No change is expected in rate limit or number of buffered records. We only expect buffering thread to read - // 2 more records into the buffer. - Assert.assertEquals(0, bufferedIterator.rateLimiter.availablePermits()); - Assert.assertEquals(recordLimit, bufferedIterator.currentRateLimit); - Assert.assertEquals(recordLimit, bufferedIterator.size()); - Assert.assertEquals(recordLimit - 1 + 2, bufferedIterator.samplingRecordCounter.get()); - } - - // Test to ensure that exception in either buffering thread or BufferedIterator-reader thread is propagated to - // another thread. - @Test(timeout = 60000) - public void testException() throws IOException, InterruptedException { - final int numRecords = 256; - final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - // buffer memory limit - final long memoryLimitInBytes = 4 * SizeEstimator.estimate(hoodieRecords.get(0)); - - // first let us throw exception from bufferIterator reader and test that buffering thread stops and throws - // correct exception back. - BufferedIterator bufferedIterator1 = - new BufferedIterator(hoodieRecords.iterator(), memoryLimitInBytes, HoodieTestDataGenerator.avroSchema); - Future result = - recordReader.submit( - () -> { - bufferedIterator1.startBuffering(); - return true; - } - ); - // waiting for permits to expire. - while (!isQueueFull(bufferedIterator1.rateLimiter)) { - Thread.sleep(10); - } - // notify buffering thread of an exception and ensure that it exits. - final Exception e = new Exception("Failing it :)"); - bufferedIterator1.markAsFailed(e); - try { - result.get(); - Assert.fail("exception is expected"); - } catch (ExecutionException e1) { - Assert.assertEquals(HoodieException.class, e1.getCause().getClass()); - Assert.assertEquals(e, e1.getCause().getCause()); - } - - // second let us raise an exception while doing record buffering. this exception should get propagated to - // buffered iterator reader. - final RuntimeException expectedException = new RuntimeException("failing record reading"); - final Iterator mockHoodieRecordsIterator = mock(Iterator.class); - when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); - when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); - BufferedIterator bufferedIterator2 = - new BufferedIterator(mockHoodieRecordsIterator, memoryLimitInBytes, HoodieTestDataGenerator.avroSchema); - Future result2 = - recordReader.submit( - () -> { - bufferedIterator2.startBuffering(); - return true; - } - ); - try { - bufferedIterator2.hasNext(); - Assert.fail("exception is expected"); - } catch (Exception e1) { - Assert.assertEquals(expectedException, e1.getCause()); - } - // buffering thread should also have exited. make sure that it is not running. - try { - result2.get(); - Assert.fail("exception is expected"); - } catch (ExecutionException e2) { - Assert.assertEquals(expectedException, e2.getCause()); - } - } - - private boolean isQueueFull(Semaphore rateLimiter) { - return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads()); - } + private boolean isQueueFull(Semaphore rateLimiter) { + return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads()); + } } 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 0316372ca..dcd5a94a5 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 @@ -55,32 +55,24 @@ public class TestUpdateMapFunction { 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); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); - String recordStr1 = - "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = - "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = - "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; List records = new ArrayList<>(); TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - records.add( - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), - rowChange1)); + records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - records.add( - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), - rowChange2)); + records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - records.add( - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), - rowChange3)); + records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); Iterator> insertResult = table.handleInsert("100", records.iterator()); - Path commitFile = - new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100")); + Path commitFile = new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100")); FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()).create(commitFile); // Now try an update with an evolved schema @@ -92,21 +84,20 @@ public class TestUpdateMapFunction { table = new HoodieCopyOnWriteTable(config, metaClient); // 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}"; + recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}"; records = new ArrayList<>(); rowChange1 = new TestRawTripPayload(recordStr1); - HoodieRecord record1 = - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), - rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); record1.setCurrentLocation(new HoodieRecordLocation("100", fileId)); records.add(record1); try { table.handleUpdate("101", fileId, records.iterator()); } catch (ClassCastException e) { - fail( - "UpdateFunction could not read records written with exampleSchema.txt using the exampleEvolvedSchema.txt"); + fail("UpdateFunction could not read records written with exampleSchema.txt using the " + + "exampleEvolvedSchema.txt"); } } 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 89f31fa2e..eb4a878f9 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 @@ -16,6 +16,12 @@ package com.uber.hoodie.index; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.atMost; +import static org.mockito.Mockito.times; + import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.HoodieTestDataGenerator; @@ -23,19 +29,16 @@ 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.table.HoodieTableMetaClient; -import com.uber.hoodie.common.table.TableFileSystemView; -import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.hbase.HBaseIndex; import com.uber.hoodie.table.HoodieTable; +import java.io.File; +import java.util.List; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; @@ -56,37 +59,26 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runners.MethodSorters; import org.mockito.Mockito; -import scala.Tuple2; - -import java.io.File; -import java.io.IOException; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.atLeast; -import static org.mockito.Mockito.atMost; -import static org.mockito.Mockito.times; /** - * Note :: HBaseTestingUtility is really flaky with issues where the HbaseMiniCluster fails to shutdown - * across tests, (see one problem here : https://issues.apache.org/jira/browse/HBASE-15835). - * Hence, the need to use MethodSorters.NAME_ASCENDING to make sure the tests run in order. Please alter - * the order of tests running carefully. + * Note :: HBaseTestingUtility is really flaky with issues where the HbaseMiniCluster fails to shutdown across tests, + * (see one problem here : https://issues.apache .org/jira/browse/HBASE-15835). Hence, the need to use + * MethodSorters.NAME_ASCENDING to make sure the tests run in order. Please alter the order of tests running carefully. */ @FixMethodOrder(MethodSorters.NAME_ASCENDING) public class TestHbaseIndex { private static JavaSparkContext jsc = null; - private String basePath = null; - private transient FileSystem fs; private static HBaseTestingUtility utility; private static Configuration hbaseConfig; private static String tableName = "test_table"; + private String basePath = null; + private transient FileSystem fs; private HoodieTableMetaClient metaClient; + public TestHbaseIndex() throws Exception { + } + @AfterClass public static void clean() throws Exception { if (jsc != null) { @@ -97,6 +89,20 @@ public class TestHbaseIndex { } } + @BeforeClass + public static void init() throws Exception { + + // Initialize HbaseMiniCluster + utility = new HBaseTestingUtility(); + utility.startMiniCluster(); + hbaseConfig = utility.getConnection().getConfiguration(); + utility.createTable(TableName.valueOf(tableName), Bytes.toBytes("_s")); + // Initialize a local spark env + SparkConf sparkConf = new SparkConf().setAppName("TestHbaseIndex").setMaster("local[1]"); + jsc = new JavaSparkContext(sparkConf); + jsc.hadoopConfiguration().addResource(utility.getConfiguration()); + } + @After public void clear() throws Exception { if (basePath != null) { @@ -112,25 +118,8 @@ public class TestHbaseIndex { basePath = folder.getRoot().getAbsolutePath(); // Initialize table metaClient = HoodieTableMetaClient - .initTableType(utility.getConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE, - tableName, HoodieTableConfig.DEFAULT_PAYLOAD_CLASS); - } - - public TestHbaseIndex() throws Exception { - } - - @BeforeClass - public static void init() throws Exception { - - // Initialize HbaseMiniCluster - utility = new HBaseTestingUtility(); - utility.startMiniCluster(); - hbaseConfig = utility.getConnection().getConfiguration(); - utility.createTable(TableName.valueOf(tableName), Bytes.toBytes("_s")); - // Initialize a local spark env - SparkConf sparkConf = new SparkConf().setAppName("TestHbaseIndex").setMaster("local[1]"); - jsc = new JavaSparkContext(sparkConf); - jsc.hadoopConfiguration().addResource(utility.getConfiguration()); + .initTableType(utility.getConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE, tableName, + HoodieTableConfig.DEFAULT_PAYLOAD_CLASS); } @Test @@ -156,7 +145,8 @@ public class TestHbaseIndex { JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); assertNoWriteErrors(writeStatues.collect()); - // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed commit + // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed + // commit javaRDD = index.tagLocation(writeRecords, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); @@ -167,8 +157,9 @@ public class TestHbaseIndex { javaRDD = index.tagLocation(writeRecords, hoodieTable); assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200); assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200); - assertTrue(javaRDD.filter(record -> (record.getCurrentLocation() != null - && record.getCurrentLocation().getCommitTime().equals(newCommitTime))).distinct().count() == 200); + assertTrue(javaRDD.filter( + record -> (record.getCurrentLocation() != null && record.getCurrentLocation().getCommitTime() + .equals(newCommitTime))).distinct().count() == 200); } @@ -208,7 +199,8 @@ public class TestHbaseIndex { // Rollback the last commit writeClient.rollback(newCommitTime); - // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled back commit + // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled + // back commit javaRDD = index.tagLocation(writeRecords, hoodieTable); assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0); @@ -302,12 +294,10 @@ public class TestHbaseIndex { } private HoodieWriteConfig.Builder getConfigBuilder() { - return HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(1, 1) - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024) - .withInlineCompaction(false).build()) - .withAutoCommit(false) + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(1, 1).withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).withInlineCompaction(false) + .build()).withAutoCommit(false) .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) .forTable("test-trip-table").withIndexConfig( HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE) 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 2fba00693..16f754a4b 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 @@ -31,16 +31,14 @@ public class TestHoodieIndex { HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder(); // Different types - HoodieWriteConfig config = clientConfigBuilder.withPath("") - .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()) - .build(); + HoodieWriteConfig config = clientConfigBuilder.withPath("").withIndexConfig( + indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()).build(); assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex); - config = clientConfigBuilder.withPath("").withIndexConfig( - indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); + config = clientConfigBuilder.withPath("") + .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build(); assertTrue(HoodieIndex.createIndex(config, null) instanceof InMemoryHashIndex); config = clientConfigBuilder.withPath("") - .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .build(); + .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); assertTrue(HoodieIndex.createIndex(config, null) 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 4bf91001f..8e649c73c 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 @@ -98,31 +98,33 @@ public class TestHoodieBloomIndex { @Test public void testLoadUUIDsInMemory() throws IOException { // Create one RDD of hoodie record - String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; - String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - HoodieRecord record1 = new HoodieRecord( - new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - HoodieRecord record2 = new HoodieRecord( - new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - HoodieRecord record3 = new HoodieRecord( - new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), + rowChange3); TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4); - HoodieRecord record4 = new HoodieRecord( - new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), + rowChange4); - JavaRDD recordRDD = jsc - .parallelize(Arrays.asList(record1, record2, record3, record4)); + JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); // Load to memory - Map> map = recordRDD - .mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())) - .groupByKey().collectAsMap(); + Map> map = recordRDD.mapToPair( + record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())).groupByKey().collectAsMap(); assertEquals(map.size(), 2); List list1 = Lists.newArrayList(map.get("2016/01/31")); List list2 = Lists.newArrayList(map.get("2015/01/31")); @@ -132,44 +134,40 @@ public class TestHoodieBloomIndex { @Test public void testLoadInvolvedFiles() throws IOException { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder() - .withPath(basePath) - .build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieBloomIndex index = new HoodieBloomIndex(config, jsc); // Create some partitions, and put some files // "2016/01/21": 0 file // "2016/04/01": 1 file (2_0_20160401010101.parquet) - // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet) + // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, + // 4_0_20150312101010.parquet) new File(basePath + "/2016/01/21").mkdirs(); new File(basePath + "/2016/04/01").mkdirs(); new File(basePath + "/2015/03/12").mkdirs(); TestRawTripPayload rowChange1 = new TestRawTripPayload( "{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); - HoodieRecord record1 = new HoodieRecord( - new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); TestRawTripPayload rowChange2 = new TestRawTripPayload( "{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); - HoodieRecord record2 = new HoodieRecord( - new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); TestRawTripPayload rowChange3 = new TestRawTripPayload( "{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); - HoodieRecord record3 = new HoodieRecord( - new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), + rowChange3); TestRawTripPayload rowChange4 = new TestRawTripPayload( "{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); - HoodieRecord record4 = new HoodieRecord( - new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), + rowChange4); - writeParquetFile("2016/04/01", "2_0_20160401010101.parquet", Lists.newArrayList(), schema, null, + writeParquetFile("2016/04/01", "2_0_20160401010101.parquet", Lists.newArrayList(), schema, null, false); + writeParquetFile("2015/03/12", "1_0_20150312101010.parquet", Lists.newArrayList(), schema, null, false); + writeParquetFile("2015/03/12", "3_0_20150312101010.parquet", Arrays.asList(record1), schema, null, false); + writeParquetFile("2015/03/12", "4_0_20150312101010.parquet", Arrays.asList(record2, record3, record4), schema, null, false); - writeParquetFile("2015/03/12", "1_0_20150312101010.parquet", Lists.newArrayList(), schema, null, - false); - writeParquetFile("2015/03/12", "3_0_20150312101010.parquet", Arrays.asList(record1), schema, - null, false); - writeParquetFile("2015/03/12", "4_0_20150312101010.parquet", - Arrays.asList(record2, record3, record4), schema, null, false); List partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); @@ -198,51 +196,32 @@ public class TestHoodieBloomIndex { List> expected = Arrays.asList( new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2_0_20160401010101.parquet")), new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1_0_20150312101010.parquet")), - new Tuple2<>("2015/03/12", - new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000")), - new Tuple2<>("2015/03/12", - new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003")) - ); + new Tuple2<>("2015/03/12", new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000")), + new Tuple2<>("2015/03/12", new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003"))); assertEquals(expected, filesList); } @Test public void testRangePruning() { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder() - .withPath(basePath) - .build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieBloomIndex index = new HoodieBloomIndex(config, jsc); final Map> partitionToFileIndexInfo = new HashMap<>(); - partitionToFileIndexInfo.put("2017/10/22", Arrays.asList( - new BloomIndexFileInfo("f1"), - new BloomIndexFileInfo("f2", "000", "000"), - new BloomIndexFileInfo("f3", "001", "003"), - new BloomIndexFileInfo("f4", "002", "007"), - new BloomIndexFileInfo("f5", "009", "010") - )); + partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"), + new BloomIndexFileInfo("f2", "000", "000"), new BloomIndexFileInfo("f3", "001", "003"), + new BloomIndexFileInfo("f4", "002", "007"), new BloomIndexFileInfo("f5", "009", "010"))); - JavaPairRDD partitionRecordKeyPairRDD = jsc - .parallelize(Arrays.asList( - new Tuple2<>("2017/10/22", "003"), - new Tuple2<>("2017/10/22", "002"), - new Tuple2<>("2017/10/22", "005"), - new Tuple2<>("2017/10/22", "004") - )) - .mapToPair(t -> t); + JavaPairRDD partitionRecordKeyPairRDD = jsc.parallelize(Arrays.asList( + new Tuple2<>("2017/10/22", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"), + new Tuple2<>("2017/10/22", "004"))).mapToPair(t -> t); - List>> comparisonKeyList = index - .explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD) - .collect(); + List>> comparisonKeyList = index.explodeRecordRDDWithFileComparisons( + partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect(); assertEquals(10, comparisonKeyList.size()); - Map> recordKeyToFileComps = comparisonKeyList.stream() - .collect(Collectors.groupingBy( - t -> t._2()._2().getRecordKey(), - Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList() - ) - )); + Map> recordKeyToFileComps = comparisonKeyList.stream().collect(Collectors.groupingBy( + t -> t._2()._2().getRecordKey(), Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList()))); assertEquals(4, recordKeyToFileComps.size()); assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("002")); @@ -252,32 +231,35 @@ public class TestHoodieBloomIndex { } @Test - public void testCheckUUIDsAgainstOneFile() - throws IOException, InterruptedException, ClassNotFoundException { + public void testCheckUUIDsAgainstOneFile() throws IOException, InterruptedException, ClassNotFoundException { // Create some records to use - String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; - String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}"; + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}"; TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - HoodieRecord record1 = new HoodieRecord( - new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - HoodieRecord record2 = new HoodieRecord( - new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - HoodieRecord record3 = new HoodieRecord( - new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), + rowChange3); TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4); - HoodieRecord record4 = new HoodieRecord( - new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), + rowChange4); - // We write record1, record2 to a parquet file, but the bloom filter contains (record1, record2, record3). + // We write record1, record2 to a parquet file, but the bloom filter contains (record1, + // record2, record3). BloomFilter filter = new BloomFilter(10000, 0.0000001); filter.add(record3.getRecordKey()); - String filename = writeParquetFile("2016/01/31", Arrays.asList(record1, record2), schema, - filter, true); + String filename = writeParquetFile("2016/01/31", Arrays.asList(record1, record2), schema, filter, true); // The bloom filter contains 3 records assertTrue(filter.mightContain(record1.getRecordKey())); @@ -286,17 +268,16 @@ public class TestHoodieBloomIndex { assertFalse(filter.mightContain(record4.getRecordKey())); // Compare with file - List uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), - record3.getRecordKey(), record4.getRecordKey()); + List uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), record3.getRecordKey(), + record4.getRecordKey()); - List results = HoodieBloomIndexCheckFunction - .checkCandidatesAgainstFile(jsc.hadoopConfiguration(), uuids, + List results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(jsc.hadoopConfiguration(), uuids, new Path(basePath + "/2016/01/31/" + filename)); assertEquals(results.size(), 2); - assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0") - || results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")); - assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0") - || results.get(1).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")); + assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0") || results.get(1).equals( + "1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")); + assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0") || results.get(1).equals( + "2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")); // TODO(vc): Need more coverage on actual filenames //assertTrue(results.get(0)._2().equals(filename)); //assertTrue(results.get(1)._2().equals(filename)); @@ -317,8 +298,7 @@ public class TestHoodieBloomIndex { try { bloomIndex.tagLocation(recordRDD, table); } catch (IllegalArgumentException e) { - fail( - "EmptyRDD should not result in IllegalArgumentException: Positive number of slices required"); + fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices " + "required"); } } @@ -327,24 +307,27 @@ public class TestHoodieBloomIndex { public void testTagLocation() throws Exception { // We have some records to be tagged (two different partitions) - String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; - String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - HoodieRecord record1 = new HoodieRecord( - new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - HoodieRecord record2 = new HoodieRecord( - new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - HoodieRecord record3 = new HoodieRecord( - new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), + rowChange3); TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4); - HoodieRecord record4 = new HoodieRecord( - new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); - JavaRDD recordRDD = jsc - .parallelize(Arrays.asList(record1, record2, record3, record4)); + HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), + rowChange4); + JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); // Also create the metadata and config HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); @@ -389,10 +372,14 @@ public class TestHoodieBloomIndex { public void testCheckExists() throws Exception { // We have some records to be tagged (two different partitions) - String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; - String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()); HoodieRecord record1 = new HoodieRecord(key1, rowChange1); @@ -414,8 +401,7 @@ public class TestHoodieBloomIndex { // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc); - JavaPairRDD> taggedRecordRDD = bloomIndex - .fetchRecordLocation(keysRDD, table); + JavaPairRDD> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table); // Should not find any files for (Tuple2> record : taggedRecordRDD.collect()) { @@ -456,16 +442,18 @@ public class TestHoodieBloomIndex { @Test public void testBloomFilterFalseError() throws IOException, InterruptedException { // We have two hoodie records - String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; // We write record1 to a parquet file, using a bloom filter having both records TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - HoodieRecord record1 = new HoodieRecord( - new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), + rowChange1); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - HoodieRecord record2 = new HoodieRecord( - new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), + rowChange2); BloomFilter filter = new BloomFilter(10000, 0.0000001); filter.add(record2.getRecordKey()); @@ -492,8 +480,8 @@ public class TestHoodieBloomIndex { } } - private String writeParquetFile(String partitionPath, List records, Schema schema, - BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException { + private String writeParquetFile(String partitionPath, List records, Schema schema, BloomFilter filter, + boolean createCommitTime) throws IOException, InterruptedException { Thread.sleep(1000); String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); String fileId = UUID.randomUUID().toString(); @@ -502,15 +490,14 @@ public class TestHoodieBloomIndex { return writeParquetFile(partitionPath, filename, records, schema, filter, createCommitTime); } - private String writeParquetFile(String partitionPath, String filename, List records, - Schema schema, + private String writeParquetFile(String partitionPath, String filename, List records, Schema schema, BloomFilter filter, boolean createCommitTime) throws IOException { if (filter == null) { filter = new BloomFilter(10000, 0.0000001); } - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( - new AvroSchemaConverter().convert(schema), schema, filter); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, + filter); String commitTime = FSUtils.getCommitTime(filename); HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, @@ -525,9 +512,7 @@ public class TestHoodieBloomIndex { for (HoodieRecord record : records) { GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++); - HoodieAvroUtils - .addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), - filename); + HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), filename); writer.writeAvro(record.getRecordKey(), avroRecord); filter.add(record.getRecordKey()); } @@ -536,9 +521,7 @@ public class TestHoodieBloomIndex { if (createCommitTime) { // Also make sure the commit is valid new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs(); - new File( - basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + ".commit") - .createNewFile(); + new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + ".commit").createNewFile(); } return filename; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java index d171a80d9..b099d8873 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCommitArchiveLog.java @@ -77,7 +77,8 @@ public class TestHoodieCommitArchiveLog { public void testArchiveDatasetWithArchival() throws IOException { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build()) + .withCompactionConfig( + HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build()) .forTable("test-trip-table").build(); HoodieTestUtils.init(hadoopConf, basePath); HoodieTestDataGenerator.createCommitFile(basePath, "100"); @@ -88,8 +89,7 @@ public class TestHoodieCommitArchiveLog { HoodieTestDataGenerator.createCommitFile(basePath, "105"); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); - HoodieTimeline timeline = - metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); @@ -103,8 +103,7 @@ public class TestHoodieCommitArchiveLog { HoodieTestUtils.createInflightCleanFiles(basePath, "106", "107"); //reload the timeline and get all the commmits before archive - timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); List originalCommits = timeline.getInstants().collect(Collectors.toList()); assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants()); @@ -118,13 +117,12 @@ public class TestHoodieCommitArchiveLog { assertTrue(archiveLog.archiveIfRequired()); //reload the timeline and remove the remaining commits - timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList())); //read the file - HoodieLogFormat.Reader reader = HoodieLogFormat - .newReader(fs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), + HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, + new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), HoodieArchivedMetaEntry.getClassSchema()); int archivedRecordsCount = 0; @@ -137,8 +135,7 @@ public class TestHoodieCommitArchiveLog { assertEquals("Archived and read records for each block are same", 8, records.size()); archivedRecordsCount += records.size(); } - assertEquals("Total archived records and total read records are the same count", 8, - archivedRecordsCount); + assertEquals("Total archived records and total read records are the same count", 8, archivedRecordsCount); //make sure the archived commits are the same as the (originalcommits - commitsleft) List readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> { @@ -146,10 +143,8 @@ public class TestHoodieCommitArchiveLog { }).collect(Collectors.toList()); Collections.sort(readCommits); - assertEquals( - "Read commits map should match the originalCommits - commitsLoadedFromArchival", - originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), - readCommits); + assertEquals("Read commits map should match the originalCommits - commitsLoadedFromArchival", + originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), readCommits); // verify in-flight instants after archive verifyInflightInstants(metaClient, 3); @@ -168,15 +163,12 @@ public class TestHoodieCommitArchiveLog { HoodieTestDataGenerator.createCommitFile(basePath, "102"); HoodieTestDataGenerator.createCommitFile(basePath, "103"); - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); - assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, - timeline.countInstants()); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); + assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants()); } @Test @@ -194,21 +186,15 @@ public class TestHoodieCommitArchiveLog { HoodieTestDataGenerator.createCommitFile(basePath, "104"); HoodieTestDataGenerator.createCommitFile(basePath, "105"); - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); - assertTrue("Archived commits should always be safe", - timeline.containsOrBeforeTimelineStarts("100")); - assertTrue("Archived commits should always be safe", - timeline.containsOrBeforeTimelineStarts("101")); - assertTrue("Archived commits should always be safe", - timeline.containsOrBeforeTimelineStarts("102")); - assertTrue("Archived commits should always be safe", - timeline.containsOrBeforeTimelineStarts("103")); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); + assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100")); + assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("101")); + assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("102")); + assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("103")); } @Test @@ -227,16 +213,14 @@ public class TestHoodieCommitArchiveLog { HoodieTestDataGenerator.createCommitFile(basePath, "104"); HoodieTestDataGenerator.createCommitFile(basePath, "105"); - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); assertEquals( - "Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)", - 5, timeline.countInstants()); + "Since we have a savepoint at 101, we should never archive any commit after 101 (we only " + "archive 100)", 5, + timeline.countInstants()); assertTrue("Archived commits should always be safe", timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101"))); assertTrue("Archived commits should always be safe", @@ -248,7 +232,7 @@ public class TestHoodieCommitArchiveLog { private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) { HoodieTimeline timeline = metaClient.getActiveTimeline().reload() .getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterInflights(); - assertEquals("Loaded inflight clean actions and the count should match", - expectedTotalInstants, timeline.countInstants()); + assertEquals("Loaded inflight clean actions and the count should match", expectedTotalInstants, + timeline.countInstants()); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java index 5a1ff7ba8..2c93f3d18 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java @@ -93,32 +93,27 @@ public class TestHoodieCompactor { } private HoodieWriteConfig.Builder getConfigBuilder() { - return HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024) - .withInlineCompaction(false).build()) - .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) - .forTable("test-trip-table").withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).withInlineCompaction(false) + .build()).withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build()) + .forTable("test-trip-table") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); } @Test(expected = IllegalArgumentException.class) public void testCompactionOnCopyOnWriteFail() throws Exception { HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); } @Test public void testCompactionEmpty() throws Exception { - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = getConfig(); - HoodieTable table = HoodieTable - .getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); String newCommitTime = writeClient.startCommit(); @@ -126,10 +121,9 @@ public class TestHoodieCompactor { JavaRDD recordsRDD = jsc.parallelize(records, 1); writeClient.insert(recordsRDD, newCommitTime).collect(); - JavaRDD result = - compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); - assertTrue("If there is nothing to compact, result will be empty", - result.isEmpty()); + JavaRDD result = compactor + .compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); + assertTrue("If there is nothing to compact, result will be empty", result.isEmpty()); } @Test @@ -145,8 +139,7 @@ public class TestHoodieCompactor { List statuses = writeClient.insert(recordsRDD, newCommitTime).collect(); // Update all the 100 records - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); newCommitTime = "101"; @@ -159,19 +152,16 @@ public class TestHoodieCompactor { // Write them to corresponding avro logfiles HoodieTestUtils - .writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, - updatedRecords); + .writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, updatedRecords); // Verify that all data file has one log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); for (String partitionPath : dataGen.getPartitionPaths()) { - List groupedLogFiles = - table.getRTFileSystemView().getLatestFileSlices(partitionPath) - .collect(Collectors.toList()); + List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) + .collect(Collectors.toList()); for (FileSlice fileSlice : groupedLogFiles) { - assertEquals("There should be 1 log file written for every data file", 1, - fileSlice.getLogFiles().count()); + assertEquals("There should be 1 log file written for every data file", 1, fileSlice.getLogFiles().count()); } } @@ -179,18 +169,19 @@ public class TestHoodieCompactor { metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); - JavaRDD result = - compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); + JavaRDD result = compactor + .compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); // Verify that all partition paths are present in the WriteStatus result for (String partitionPath : dataGen.getPartitionPaths()) { List writeStatuses = result.collect(); assertTrue(writeStatuses.stream() - .filter(writeStatus -> writeStatus.getStat().getPartitionPath() - .contentEquals(partitionPath)).count() > 0); + .filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)) + .count() > 0); } } - // TODO - after modifying HoodieReadClient to support realtime tables - add more tests to make sure the data read is the updated data (compaction correctness) + // TODO - after modifying HoodieReadClient to support realtime tables - add more tests to make + // sure the data read is the updated data (compaction correctness) // TODO - add more test cases for compactions after a failed commit/compaction } 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 55898c373..c3aee4f2b 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 @@ -16,6 +16,9 @@ package com.uber.hoodie.io.strategy; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import com.beust.jcommander.internal.Lists; import com.google.common.collect.Maps; import com.uber.hoodie.config.HoodieCompactionConfig; @@ -25,20 +28,16 @@ import com.uber.hoodie.io.compact.strategy.BoundedIOCompactionStrategy; import com.uber.hoodie.io.compact.strategy.DayBasedCompactionStrategy; import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy; import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy; -import org.junit.Test; - import java.util.List; import java.util.Map; import java.util.Random; import java.util.stream.Collectors; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.junit.Test; public class TestHoodieCompactionStrategy { private static final long MB = 1024 * 1024L; - private String [] partitionPaths = {"2017/01/01", "2017/01/02", "2017/01/03"}; + private String[] partitionPaths = {"2017/01/01", "2017/01/02", "2017/01/03"}; @Test public void testUnBounded() { @@ -48,9 +47,8 @@ public class TestHoodieCompactionStrategy { sizesMap.put(100 * MB, Lists.newArrayList(MB)); sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); UnBoundedCompactionStrategy strategy = new UnBoundedCompactionStrategy(); - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp") - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).build()).build(); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( + HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).build()).build(); List operations = createCompactionOperations(writeConfig, sizesMap); List returned = strategy.orderAndFilter(writeConfig, operations); assertEquals("UnBounded should not re-order or filter", operations, returned); @@ -64,23 +62,19 @@ public class TestHoodieCompactionStrategy { sizesMap.put(100 * MB, Lists.newArrayList(MB)); sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); BoundedIOCompactionStrategy strategy = new BoundedIOCompactionStrategy(); - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp") - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy) - .withTargetIOPerCompactionInMB(400).build()).build(); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( + HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build()) + .build(); List operations = createCompactionOperations(writeConfig, sizesMap); List returned = strategy.orderAndFilter(writeConfig, operations); - assertTrue("BoundedIOCompaction should have resulted in fewer compactions", - returned.size() < operations.size()); - assertEquals("BoundedIOCompaction should have resulted in 2 compactions being chosen", - 2, returned.size()); + assertTrue("BoundedIOCompaction should have resulted in fewer compactions", returned.size() < operations.size()); + assertEquals("BoundedIOCompaction should have resulted in 2 compactions being chosen", 2, returned.size()); // Total size of all the log files - Long returnedSize = returned.stream() - .map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)).map(s -> (Long) s) - .reduce((size1, size2) -> size1 + size2).orElse(0L); - assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", - 610, (long) returnedSize); + Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)) + .map(s -> (Long) s).reduce((size1, size2) -> size1 + size2).orElse(0L); + assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 610, + (long) returnedSize); } @Test @@ -91,23 +85,20 @@ public class TestHoodieCompactionStrategy { sizesMap.put(100 * MB, Lists.newArrayList(MB)); sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); LogFileSizeBasedCompactionStrategy strategy = new LogFileSizeBasedCompactionStrategy(); - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp") - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy) - .withTargetIOPerCompactionInMB(400).build()).build(); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( + HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build()) + .build(); List operations = createCompactionOperations(writeConfig, sizesMap); List returned = strategy.orderAndFilter(writeConfig, operations); assertTrue("LogFileSizeBasedCompactionStrategy should have resulted in fewer compactions", returned.size() < operations.size()); - assertEquals("LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction", - 1, returned.size()); + assertEquals("LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction", 1, returned.size()); // Total size of all the log files - Long returnedSize = returned.stream() - .map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)).map(s -> (Long) s) - .reduce((size1, size2) -> size1 + size2).orElse(0L); - assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", - 1204, (long) returnedSize); + Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)) + .map(s -> (Long) s).reduce((size1, size2) -> size1 + size2).orElse(0L); + assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 1204, + (long) returnedSize); } @Test @@ -118,20 +109,18 @@ public class TestHoodieCompactionStrategy { sizesMap.put(100 * MB, Lists.newArrayList(MB)); sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB)); DayBasedCompactionStrategy strategy = new DayBasedCompactionStrategy(); - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp") - .withCompactionConfig( - HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy) - .withTargetIOPerCompactionInMB(400).build()).build(); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig( + HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build()) + .build(); List operations = createCompactionOperations(writeConfig, sizesMap); List returned = strategy.orderAndFilter(writeConfig, operations); assertTrue("DayBasedCompactionStrategy should have resulted in fewer compactions", - returned.size() < operations.size()); + returned.size() < operations.size()); int comparision = strategy.getComparator().compare(returned.get(returned.size() - 1), returned.get(0)); // Either the partition paths are sorted in descending order or they are equal - assertTrue("DayBasedCompactionStrategy should sort partitions in descending order", - comparision >= 0); + assertTrue("DayBasedCompactionStrategy should sort partitions in descending order", comparision >= 0); } private List createCompactionOperations(HoodieWriteConfig config, @@ -139,9 +128,8 @@ public class TestHoodieCompactionStrategy { List operations = Lists.newArrayList(sizesMap.size()); sizesMap.forEach((k, v) -> { operations.add(new CompactionOperation(TestHoodieDataFile.newDataFile(k), - partitionPaths[new Random().nextInt(partitionPaths.length - 1)], - v.stream().map(TestHoodieLogFile::newLogFile).collect( - Collectors.toList()), config)); + partitionPaths[new Random().nextInt(partitionPaths.length - 1)], + v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), config)); }); return operations; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java index 564d95218..88058b3e3 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java @@ -28,6 +28,10 @@ public class TestHoodieDataFile extends HoodieDataFile { this.size = size; } + public static HoodieDataFile newDataFile(long size) { + return new TestHoodieDataFile(size); + } + @Override public String getPath() { return "/tmp/test"; @@ -43,13 +47,8 @@ public class TestHoodieDataFile extends HoodieDataFile { return "100"; } - @Override public long getFileSize() { return size; } - - public static HoodieDataFile newDataFile(long size) { - return new TestHoodieDataFile(size); - } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java index d23cbf27c..df46ec214 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java @@ -29,6 +29,10 @@ public class TestHoodieLogFile extends HoodieLogFile { this.size = size; } + public static HoodieLogFile newLogFile(long size) { + return new TestHoodieLogFile(size); + } + @Override public Path getPath() { return new Path("/tmp/test-log"); @@ -38,8 +42,4 @@ public class TestHoodieLogFile extends HoodieLogFile { public Optional getFileSize() { return Optional.of(size); } - - public static HoodieLogFile newLogFile(long size) { - return new TestHoodieLogFile(size); - } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java b/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java index 911f97417..cb1a43969 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/metrics/TestHoodieMetrics.java @@ -40,7 +40,6 @@ public class TestHoodieMetrics { @Test public void testRegisterGauge() { metrics.registerGauge("metric1", 123L); - assertTrue(Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString() - .equals("123")); + assertTrue(Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString().equals("123")); } } 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 ab9ba8be4..ebd31b25a 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 @@ -89,14 +89,13 @@ public class TestCopyOnWriteTable { String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieWriteConfig config = makeHoodieClientConfig(); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath); Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName); - assertTrue(newPath.toString().equals(this.basePath + "/" + partitionPath + "/" + FSUtils - .makeDataFileName(commitTime, unitNumber, fileName))); + assertTrue(newPath.toString().equals( + this.basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, unitNumber, fileName))); } private HoodieWriteConfig makeHoodieClientConfig() throws Exception { @@ -105,8 +104,7 @@ public class TestCopyOnWriteTable { private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception { // Prepare the AvroParquetIO - String schemaStr = IOUtils - .toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8"); + String schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8"); return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr); } @@ -122,28 +120,27 @@ public class TestCopyOnWriteTable { HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); // Get some records belong to the same partition (2016/01/31) - String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; - String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}"; + String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}"; List records = new ArrayList<>(); TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - records.add( - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), - rowChange1)); + records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - records.add( - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), - rowChange2)); + records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - records.add( - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), - rowChange3)); + records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records HoodieClientTestUtils.collectStatuses(table.handleInsert(firstCommitTime, records.iterator())); - // We should have a parquet file generated (TODO: better control # files after we revise AvroParquetIO) + // We should have a parquet file generated (TODO: better control # files after we revise + // AvroParquetIO) File parquetFile = null; for (File file : new File(this.basePath + partitionPath).listFiles()) { if (file.getName().endsWith(".parquet")) { @@ -155,18 +152,17 @@ public class TestCopyOnWriteTable { // Read out the bloom filter and make sure filter can answer record exist or not Path parquetFilePath = new Path(parquetFile.getAbsolutePath()); - BloomFilter filter = ParquetUtils - .readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath); + BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath); for (HoodieRecord record : records) { assertTrue(filter.mightContain(record.getRecordKey())); } // Create a commit file - new File(this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile(); + new File( + this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + FSUtils.getCommitTime(parquetFile.getName()) + + ".commit").createNewFile(); // Read the parquet file, check the record content - List fileRecords = ParquetUtils - .readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath); + List fileRecords = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath); GenericRecord newRecord; int index = 0; for (GenericRecord record : fileRecords) { @@ -175,13 +171,12 @@ public class TestCopyOnWriteTable { } // We update the 1st record & add a new record - String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; TestRawTripPayload updateRowChanges1 = new TestRawTripPayload(updateRecordStr1); HoodieRecord updatedRecord1 = new HoodieRecord( - new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), - updateRowChanges1); - updatedRecord1.setCurrentLocation( - new HoodieRecordLocation(null, FSUtils.getFileId(parquetFile.getName()))); + new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1); + updatedRecord1.setCurrentLocation(new HoodieRecordLocation(null, FSUtils.getFileId(parquetFile.getName()))); TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4); HoodieRecord insertedRecord1 = new HoodieRecord( @@ -201,10 +196,9 @@ public class TestCopyOnWriteTable { File updatedParquetFile = null; for (File file : new File(basePath + "/2016/01/31").listFiles()) { if (file.getName().endsWith(".parquet")) { - if (FSUtils.getFileId(file.getName()) - .equals(FSUtils.getFileId(parquetFile.getName())) && - HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(file.getName()), - FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) { + if (FSUtils.getFileId(file.getName()).equals(FSUtils.getFileId(parquetFile.getName())) + && HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(file.getName()), + FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) { updatedParquetFile = file; break; } @@ -213,8 +207,8 @@ public class TestCopyOnWriteTable { assertTrue(updatedParquetFile != null); // Check whether the record has been updated Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath()); - BloomFilter updatedFilter = ParquetUtils - .readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), updatedParquetFilePath); + BloomFilter updatedFilter = ParquetUtils.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), + updatedParquetFilePath); for (HoodieRecord record : records) { // No change to the _row_key assertTrue(updatedFilter.mightContain(record.getRecordKey())); @@ -223,8 +217,7 @@ public class TestCopyOnWriteTable { assertTrue(updatedFilter.mightContain(insertedRecord1.getRecordKey())); records.add(insertedRecord1);// add this so it can further check below - ParquetReader updatedReader = ParquetReader - .builder(new AvroReadSupport<>(), updatedParquetFilePath).build(); + ParquetReader updatedReader = ParquetReader.builder(new AvroReadSupport<>(), updatedParquetFilePath).build(); index = 0; while ((newRecord = (GenericRecord) updatedReader.read()) != null) { assertTrue(newRecord.get("_row_key").toString().equals(records.get(index).getRecordKey())); @@ -246,13 +239,9 @@ public class TestCopyOnWriteTable { List records = new ArrayList<>(); for (int i = 0; i < n; i++) { String recordStr = String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}", - UUID.randomUUID().toString(), - time, - i); + UUID.randomUUID().toString(), time, i); TestRawTripPayload rowChange = new TestRawTripPayload(recordStr); - records.add(new HoodieRecord( - new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), - rowChange)); + records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); } return records; } @@ -261,31 +250,28 @@ public class TestCopyOnWriteTable { @Test public void testMetadataAggregateFromWriteStatus() throws Exception { // Prepare the AvroParquetIO - HoodieWriteConfig config = makeHoodieClientConfigBuilder() - .withWriteStatusClass(MetadataMergeWriteStatus.class).build(); + HoodieWriteConfig config = makeHoodieClientConfigBuilder().withWriteStatusClass(MetadataMergeWriteStatus.class) + .build(); String firstCommitTime = HoodieTestUtils.makeNewCommitTime(); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); // Get some records belong to the same partition (2016/01/31) - String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; - String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; - String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; List records = new ArrayList<>(); TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1); - records.add( - new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), - rowChange1)); + records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1)); TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2); - records.add( - new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), - rowChange2)); + records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2)); TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3); - records.add( - new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), - rowChange3)); + records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3)); // Insert new records List writeStatuses = HoodieClientTestUtils @@ -293,7 +279,8 @@ public class TestCopyOnWriteTable { Map allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus .mergeMetadataForWriteStatuses(writeStatuses); assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000")); - // For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this should be 2 * 3 + // For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this + // should be 2 * 3 assertEquals("6", allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000")); } @@ -314,26 +301,19 @@ public class TestCopyOnWriteTable { List statuses = HoodieClientTestUtils .collectStatuses(table.handleInsert(commitTime, records.iterator())); WriteStatus status = statuses.get(0); - Path partialFile = new Path(String.format("%s/%s/%s", - basePath, - status.getPartitionPath(), - FSUtils.makeDataFileName(commitTime, 0, status.getFileId())) - ); + Path partialFile = new Path(String.format("%s/%s/%s", basePath, status.getPartitionPath(), + FSUtils.makeDataFileName(commitTime, 0, status.getFileId()))); assertTrue(fs.exists(partialFile)); // When we retry records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z"); records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z")); - statuses = HoodieClientTestUtils - .collectStatuses(table.handleInsert(commitTime, records.iterator())); + statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator())); status = statuses.get(0); - Path retriedFIle = new Path(String.format("%s/%s/%s", - basePath, - status.getPartitionPath(), - FSUtils.makeDataFileName(commitTime, 0, status.getFileId())) - ); + Path retriedFIle = new Path(String.format("%s/%s/%s", basePath, status.getPartitionPath(), + FSUtils.makeDataFileName(commitTime, 0, status.getFileId()))); assertTrue(fs.exists(retriedFIle)); assertFalse(fs.exists(partialFile)); } @@ -371,8 +351,7 @@ public class TestCopyOnWriteTable { records.addAll(newHoodieRecords(1, "2016-02-02T03:16:41.415Z")); // Insert new records - returnedStatuses = HoodieClientTestUtils - .collectStatuses(table.handleInsert(commitTime, records.iterator())); + returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator())); assertEquals(3, returnedStatuses.size()); assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath()); @@ -389,8 +368,8 @@ public class TestCopyOnWriteTable { @Test public void testFileSizeUpsertRecords() throws Exception { HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig( - HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024) - .parquetPageSize(64 * 1024).build()).build(); + HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024) + .build()).build(); String commitTime = HoodieTestUtils.makeNewCommitTime(); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); @@ -398,12 +377,11 @@ public class TestCopyOnWriteTable { List records = new ArrayList<>(); // Approx 1150 records are written for block size of 64KB for (int i = 0; i < 2000; i++) { - String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString() - + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}"; + String recordStr = + "{\"_row_key\":\"" + UUID.randomUUID().toString() + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + + "}"; TestRawTripPayload rowChange = new TestRawTripPayload(recordStr); - records - .add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), - rowChange)); + records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange)); } // Insert new records @@ -412,39 +390,30 @@ public class TestCopyOnWriteTable { // Check the updated file int counts = 0; for (File file : new File(basePath + "/2016/01/31").listFiles()) { - if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()) - .equals(commitTime)) { + if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()).equals(commitTime)) { System.out.println(file.getName() + "-" + file.length()); counts++; } } - assertEquals( - "If the number of records are more than 1150, then there should be a new file", 3, - counts); + assertEquals("If the number of records are more than 1150, then there should be a new file", 3, counts); } - private List testUpsertPartitioner(int smallFileSize, - int numInserts, - int numUpdates, - int fileSize, - boolean autoSplitInserts) throws Exception { - final String TEST_PARTITION_PATH = "2016/09/26"; - HoodieWriteConfig config = makeHoodieClientConfigBuilder() - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .compactionSmallFileSize(smallFileSize).insertSplitSize(100) - .autoTuneInsertSplits(autoSplitInserts).build()) - .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()) - .build(); + private List testUpsertPartitioner(int smallFileSize, int numInserts, + int numUpdates, int fileSize, boolean autoSplitInserts) throws Exception { + final String testPartitionPath = "2016/09/26"; + HoodieWriteConfig config = makeHoodieClientConfigBuilder().withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(smallFileSize).insertSplitSize(100) + .autoTuneInsertSplits(autoSplitInserts).build()).withStorageConfig( + HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build(); HoodieClientTestUtils.fakeCommitFile(basePath, "001"); - HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize); + HoodieClientTestUtils.fakeDataFile(basePath, testPartitionPath, "001", "file1", fileSize); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); - HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator( - new String[]{TEST_PARTITION_PATH}); + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath}); List insertRecords = dataGenerator.generateInserts("001", numInserts); List updateRecords = dataGenerator.generateUpdates("001", numUpdates); for (HoodieRecord updateRec : updateRecords) { @@ -454,8 +423,8 @@ public class TestCopyOnWriteTable { records.addAll(insertRecords); records.addAll(updateRecords); WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records)); - HoodieCopyOnWriteTable.UpsertPartitioner partitioner = (HoodieCopyOnWriteTable.UpsertPartitioner) - table.getUpsertPartitioner(profile); + HoodieCopyOnWriteTable.UpsertPartitioner partitioner = + (HoodieCopyOnWriteTable.UpsertPartitioner) table.getUpsertPartitioner(profile); assertEquals("Should have 3 partitions", 3, partitioner.numPartitions()); assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE, @@ -464,40 +433,35 @@ public class TestCopyOnWriteTable { partitioner.getBucketInfo(1).bucketType); assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT, partitioner.getBucketInfo(2).bucketType); - assertEquals("Update record should have gone to the 1 update partiton", 0, - partitioner.getPartition(new Tuple2<>(updateRecords.get(0).getKey(), - Option.apply(updateRecords.get(0).getCurrentLocation())))); - return partitioner.getInsertBuckets(TEST_PARTITION_PATH); + assertEquals("Update record should have gone to the 1 update partiton", 0, partitioner.getPartition( + new Tuple2<>(updateRecords.get(0).getKey(), Option.apply(updateRecords.get(0).getCurrentLocation())))); + return partitioner.getInsertBuckets(testPartitionPath); } @Test public void testUpsertPartitioner() throws Exception { // Inserts + Updates... Check all updates go together & inserts subsplit - List insertBuckets = testUpsertPartitioner(0, 200, 100, - 1024, false); + List insertBuckets = testUpsertPartitioner(0, 200, 100, 1024, false); assertEquals("Total of 2 insert buckets", 2, insertBuckets.size()); } @Test public void testUpsertPartitionerWithSmallInsertHandling() throws Exception { - // Inserts + Updates .. Check updates go together & inserts subsplit, after expanding smallest file - List insertBuckets = testUpsertPartitioner(1000 * 1024, - 400, 100, 800 * 1024, false); + // Inserts + Updates .. Check updates go together & inserts subsplit, after expanding + // smallest file + List insertBuckets = testUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024, + false); assertEquals("Total of 3 insert buckets", 3, insertBuckets.size()); - assertEquals("First insert bucket must be same as update bucket", 0, - insertBuckets.get(0).bucketNumber); - assertEquals("First insert bucket should have weight 0.5", 0.5, insertBuckets.get(0).weight, - 0.01); + assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber); + assertEquals("First insert bucket should have weight 0.5", 0.5, insertBuckets.get(0).weight, 0.01); // Now with insert split size auto tuned insertBuckets = testUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, true); assertEquals("Total of 3 insert buckets", 3, insertBuckets.size()); - assertEquals("First insert bucket must be same as update bucket", 0, - insertBuckets.get(0).bucketNumber); - assertEquals("First insert bucket should have weight 0.5", 200.0 / 2400, - insertBuckets.get(0).weight, 0.01); + assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber); + assertEquals("First insert bucket should have weight 0.5", 200.0 / 2400, insertBuckets.get(0).weight, 0.01); } @After 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 23f201b45..67f8d6d6e 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 @@ -18,7 +18,6 @@ package com.uber.hoodie.table; - import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -76,15 +75,14 @@ import org.junit.rules.TemporaryFolder; public class TestMergeOnReadTable { - private transient JavaSparkContext jsc = null; - private transient SQLContext sqlContext; private static String basePath = null; - //NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class) //The implementation and gurantees of many API's differ, for example check rename(src,dst) private static MiniDFSCluster dfsCluster; private static DistributedFileSystem dfs; private static HdfsTestService hdfsTestService; + private transient JavaSparkContext jsc = null; + private transient SQLContext sqlContext; @AfterClass public static void cleanUp() throws Exception { @@ -92,13 +90,15 @@ public class TestMergeOnReadTable { hdfsTestService.stop(); dfsCluster.shutdown(); } - // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the + // same JVM FileSystem.closeAll(); } @BeforeClass public static void setUpDFS() throws IOException { - // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the + // same JVM FileSystem.closeAll(); if (hdfsTestService == null) { hdfsTestService = new HdfsTestService(); @@ -111,8 +111,7 @@ public class TestMergeOnReadTable { @Before public void init() throws IOException { // Initialize a local spark env - jsc = new JavaSparkContext( - HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeOnReadTable")); + jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeOnReadTable")); // Create a temp folder as the base path TemporaryFolder folder = new TemporaryFolder(); @@ -154,28 +153,23 @@ public class TestMergeOnReadTable { List statuses = client.upsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - cfg.getBasePath()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - Optional deltaCommit = - metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = - metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - FileStatus[] allFiles = HoodieTestUtils - .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); @@ -209,21 +203,17 @@ public class TestMergeOnReadTable { client.compact(compactionCommitTime); allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue(dataFilesToRead.findAny().isPresent()); // verify that there is a commit HoodieTable table = HoodieTable.getHoodieTable( - new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), - getConfig(false)); + new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), getConfig(false)); HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants(); - assertEquals("Expecting a single commit.", 1, - timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); + assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp(); - assertTrue(HoodieTimeline - .compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER)); + assertTrue(HoodieTimeline.compareTimestamps("000", latestCompactionCommitTime, HoodieTimeline.LESSER)); assertEquals("Must contain 200 records", 200, HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count()); @@ -232,8 +222,7 @@ public class TestMergeOnReadTable { // Check if record level metadata is aggregated properly at the end of write. @Test public void testMetadataAggregateFromWriteStatus() throws Exception { - HoodieWriteConfig cfg = getConfigBuilder(false) - .withWriteStatusClass(MetadataMergeWriteStatus.class).build(); + HoodieWriteConfig cfg = getConfigBuilder(false).withWriteStatusClass(MetadataMergeWriteStatus.class).build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); String newCommitTime = "001"; @@ -248,7 +237,8 @@ public class TestMergeOnReadTable { Map allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus .mergeMetadataForWriteStatuses(statuses); assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000")); - // For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this should be 2 * records.size() + // For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this + // should be 2 * records.size() assertEquals(String.valueOf(2 * records.size()), allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000")); } @@ -271,28 +261,23 @@ public class TestMergeOnReadTable { List statuses = client.upsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - cfg.getBasePath()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - Optional deltaCommit = - metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = - metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - FileStatus[] allFiles = HoodieTestUtils - .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); @@ -329,15 +314,12 @@ public class TestMergeOnReadTable { assertFalse(commit.isPresent()); allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue(dataFilesToRead.findAny().isPresent()); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) - .collect(Collectors.toList()); - List recordsRead = HoodieMergeOnReadTestUtils - .getRecordsUsingInputFormat(dataFiles, basePath); + List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); //Wrote 40 records and deleted 20 records, so remaining 40-20 = 20 assertEquals("Must contain 20 records", 20, recordsRead.size()); } @@ -365,10 +347,8 @@ public class TestMergeOnReadTable { //verify there are no errors assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - cfg.getBasePath()); - Optional commit = - metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); + Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertTrue(commit.isPresent()); assertEquals("commit should be 001", "001", commit.get().getTimestamp()); @@ -391,10 +371,8 @@ public class TestMergeOnReadTable { client.rollback(newCommitTime); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - HoodieTable hoodieTable = HoodieTable - .getHoodieTable(metaClient, cfg); - FileStatus[] allFiles = HoodieTestUtils - .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); @@ -428,28 +406,23 @@ public class TestMergeOnReadTable { List statuses = client.upsert(writeRecords, newCommitTime).collect(); assertNoWriteErrors(statuses); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - cfg.getBasePath()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - Optional deltaCommit = - metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = - metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - FileStatus[] allFiles = HoodieTestUtils - .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); @@ -473,10 +446,8 @@ public class TestMergeOnReadTable { commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) - .collect(Collectors.toList()); - List recordsRead = HoodieMergeOnReadTestUtils - .getRecordsUsingInputFormat(dataFiles, basePath); + List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); @@ -485,8 +456,7 @@ public class TestMergeOnReadTable { metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); @@ -512,11 +482,10 @@ public class TestMergeOnReadTable { 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); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); - final String compactedCommitTime = metaClient.getActiveTimeline().reload() - .getCommitsTimeline().lastInstant().get().getTimestamp(); + final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get() + .getTimestamp(); assertTrue(roView.getLatestDataFiles().filter(file -> { if (compactedCommitTime.equals(file.getCommitTime())) { @@ -531,8 +500,7 @@ public class TestMergeOnReadTable { 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); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), allFiles); assertFalse(roView.getLatestDataFiles().filter(file -> { if (compactedCommitTime.equals(file.getCommitTime())) { @@ -564,30 +532,28 @@ public class TestMergeOnReadTable { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); - Optional deltaCommit = - metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = - metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - FileStatus[] allFiles = HoodieTestUtils - .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); + FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = roView.getLatestDataFiles(); - Map parquetFileIdToSize = dataFilesToRead.collect(Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); + Map parquetFileIdToSize = dataFilesToRead.collect( + Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), - allFiles); + roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); /** - * Write 2 (only updates + inserts, written to .log file + correction of existing parquet file size) + * Write 2 (only updates + inserts, written to .log file + correction of existing parquet + * file size) */ newCommitTime = "002"; client.startCommitWithTime(newCommitTime); @@ -608,18 +574,17 @@ public class TestMergeOnReadTable { assertFalse(commit.isPresent()); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); - roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getActiveTimeline().reload() - .getCommitsTimeline().filterCompletedInstants(), allFiles); + roView = new HoodieTableFileSystemView(metaClient, + hoodieTable.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); - Map parquetFileIdToNewSize = dataFilesToRead.collect(Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); + Map parquetFileIdToNewSize = dataFilesToRead.collect( + Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); assertTrue(parquetFileIdToNewSize.entrySet().stream() .filter(entry -> parquetFileIdToSize.get(entry.getKey()) < entry.getValue()).count() > 0); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) - .collect(Collectors.toList()); - List recordsRead = HoodieMergeOnReadTestUtils - .getRecordsUsingInputFormat(dataFiles, basePath); + List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); //Wrote 20 records in 2 batches assertEquals("Must contain 40 records", 40, recordsRead.size()); } @@ -639,8 +604,7 @@ public class TestMergeOnReadTable { List statuses = writeClient.insert(recordsRDD, newCommitTime).collect(); // Update all the 100 records - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), - basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); newCommitTime = "101"; @@ -653,19 +617,17 @@ public class TestMergeOnReadTable { // Write them to corresponding avro logfiles HoodieTestUtils - .writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(), - HoodieTestDataGenerator.avroSchema, updatedRecords); + .writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, + updatedRecords); // Verify that all data file has one log file metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); table = HoodieTable.getHoodieTable(metaClient, config); for (String partitionPath : dataGen.getPartitionPaths()) { - List groupedLogFiles = - table.getRTFileSystemView().getLatestFileSlices(partitionPath) - .collect(Collectors.toList()); + List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) + .collect(Collectors.toList()); for (FileSlice fileSlice : groupedLogFiles) { - assertEquals("There should be 1 log file written for every data file", 1, - fileSlice.getLogFiles().count()); + assertEquals("There should be 1 log file written for every data file", 1, fileSlice.getLogFiles().count()); } } @@ -674,31 +636,27 @@ public class TestMergeOnReadTable { table = HoodieTable.getHoodieTable(metaClient, config); String commitTime = writeClient.startCompaction(); - JavaRDD result = - writeClient.compact(commitTime); + 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); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - assertTrue("Compaction commit should be > than last insert", - HoodieTimeline.compareTimestamps(timeline.lastInstant().get().getTimestamp(), newCommitTime, - HoodieTimeline.GREATER)); + assertTrue("Compaction commit should be > than last insert", HoodieTimeline.compareTimestamps( + timeline.lastInstant().get().getTimestamp(), newCommitTime, HoodieTimeline.GREATER)); for (String partitionPath : dataGen.getPartitionPaths()) { - List groupedLogFiles = table.getRTFileSystemView() - .getLatestFileSlices(partitionPath) + List groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath) .collect(Collectors.toList()); for (FileSlice slice : groupedLogFiles) { - assertTrue( - "After compaction there should be no log files visiable on a Realtime view", + assertTrue("After compaction there should be no log files visiable on a Realtime view", slice.getLogFiles().collect(Collectors.toList()).isEmpty()); } List writeStatuses = result.collect(); assertTrue(writeStatuses.stream() - .filter(writeStatus -> writeStatus.getStat().getPartitionPath() - .contentEquals(partitionPath)).count() > 0); + .filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)) + .count() > 0); } } @@ -707,16 +665,13 @@ public class TestMergeOnReadTable { } private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) { - 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()) + 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()); + .forTable("test-trip-table") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()); } private void assertNoWriteErrors(List statuses) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java b/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java index f793c1539..65e713335 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java @@ -33,10 +33,10 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport { private String maxRecordKey; - public final static String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY = + public static final String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY = "com.uber.hoodie.bloomfilter"; - public final static String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key"; - public final static String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key"; + public static final String HOODIE_MIN_RECORD_KEY_FOOTER = "hoodie_min_record_key"; + public static final String HOODIE_MAX_RECORD_KEY_FOOTER = "hoodie_max_record_key"; public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java index 827da3093..b0e06c028 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java @@ -16,6 +16,7 @@ package com.uber.hoodie.avro; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -24,11 +25,11 @@ import java.util.Map; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; -import com.fasterxml.jackson.databind.ObjectMapper; /** - * Marjority of this is copied from https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/common/JsonConverter.java - * Adjusted for expected behavior of our use cases + * Marjority of this is copied from + * https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/ + * common/JsonConverter.java Adjusted for expected behavior of our use cases */ public class MercifulJsonConverter { @@ -132,10 +133,10 @@ public class MercifulJsonConverter { } private boolean isOptional(Schema schema) { - return schema.getType().equals(Schema.Type.UNION) && - schema.getTypes().size() == 2 && - (schema.getTypes().get(0).getType().equals(Schema.Type.NULL) || - schema.getTypes().get(1).getType().equals(Schema.Type.NULL)); + return schema.getType().equals(Schema.Type.UNION) + && schema.getTypes().size() == 2 + && (schema.getTypes().get(0).getType().equals(Schema.Type.NULL) + || schema.getTypes().get(1).getType().equals(Schema.Type.NULL)); } private Schema getNonNull(Schema schema) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java index d2de837ef..d38a2a660 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java @@ -113,9 +113,8 @@ public class HoodieCleanStat implements Serializable { } public Builder withEarliestCommitRetained(Optional earliestCommitToRetain) { - this.earliestCommitToRetain = (earliestCommitToRetain.isPresent()) ? - earliestCommitToRetain.get().getTimestamp() : - "-1"; + this.earliestCommitToRetain = (earliestCommitToRetain.isPresent()) + ? earliestCommitToRetain.get().getTimestamp() : "-1"; return this; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java index 21c1816f5..3f1b69080 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java @@ -210,12 +210,18 @@ public class HoodieCommitMetadata implements Serializable { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } HoodieCommitMetadata that = (HoodieCommitMetadata) o; - if (!partitionToWriteStats.equals(that.partitionToWriteStats)) return false; + if (!partitionToWriteStats.equals(that.partitionToWriteStats)) { + return false; + } return compacted.equals(that.compacted); } 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 d5884fb99..f3111a109 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 @@ -39,7 +39,6 @@ public class HoodieFileGroup implements Serializable { }; } - /** * Partition containing the file group. */ @@ -107,10 +106,10 @@ public class HoodieFileGroup implements Serializable { */ private boolean isFileSliceCommitted(FileSlice slice) { String maxCommitTime = lastInstant.get().getTimestamp(); - return timeline.containsOrBeforeTimelineStarts(slice.getBaseCommitTime()) && - HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(), - maxCommitTime, - HoodieTimeline.LESSER_OR_EQUAL); + return timeline.containsOrBeforeTimelineStarts(slice.getBaseCommitTime()) + && HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(), + maxCommitTime, + HoodieTimeline.LESSER_OR_EQUAL); } @@ -128,7 +127,7 @@ public class HoodieFileGroup implements Serializable { /** * Gets the latest slice - this can contain either - * + *

* - just the log files without data file - (or) data file with 0 or more log files */ public Optional getLatestFileSlice() { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java index f4545809c..0581a5a20 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java @@ -21,7 +21,7 @@ import java.io.Serializable; /** * HoodieKey consists of - * + *

* - recordKey : a recordKey that acts as primary key for a record - partitionPath : path to the * partition that contains the record */ @@ -54,8 +54,8 @@ public class HoodieKey implements Serializable { return false; } HoodieKey otherKey = (HoodieKey) o; - return Objects.equal(recordKey, otherKey.recordKey) && - Objects.equal(partitionPath, otherKey.partitionPath); + return Objects.equal(recordKey, otherKey.recordKey) + && Objects.equal(partitionPath, otherKey.partitionPath); } @Override 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 0c587c671..7dfaf0bdf 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 @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.Path; /** * Abstracts a single log file. Contains methods to extract metadata like the fileId, version and * extension from the log file path. - * + *

* Also contains logic to roll-over the log file */ public class HoodieLogFile implements Serializable { @@ -103,8 +103,12 @@ public class HoodieLogFile implements Serializable { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } HoodieLogFile that = (HoodieLogFile) o; return path != null ? path.equals(that.path) : that.path == null; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java index ebe801811..960042a84 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodiePartitionMetadata.java @@ -101,9 +101,8 @@ public class HoodiePartitionMetadata { } } catch (IOException ioe) { log.warn( - "Error trying to save partition metadata (this is okay, as long as atleast 1 of these succced), " - + - partitionPath, ioe); + "Error trying to save partition metadata (this is okay, as long as " + + "atleast 1 of these succced), " + partitionPath, ioe); } finally { if (!metafileExists) { try { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java index 43b023020..16d75491a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java @@ -118,10 +118,10 @@ public class HoodieRecord implements Serializable return false; } HoodieRecord that = (HoodieRecord) o; - return Objects.equal(key, that.key) && - Objects.equal(data, that.data) && - Objects.equal(currentLocation, that.currentLocation) && - Objects.equal(newLocation, that.newLocation); + return Objects.equal(key, that.key) + && Objects.equal(data, that.data) + && Objects.equal(currentLocation, that.currentLocation) + && Objects.equal(newLocation, that.newLocation); } @Override diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java index fa5f7a04f..51edf4951 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java @@ -42,8 +42,8 @@ public class HoodieRecordLocation implements Serializable { return false; } HoodieRecordLocation otherLoc = (HoodieRecordLocation) o; - return Objects.equal(commitTime, otherLoc.commitTime) && - Objects.equal(fileId, otherLoc.fileId); + return Objects.equal(commitTime, otherLoc.commitTime) + && Objects.equal(fileId, otherLoc.fileId); } @Override diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java index c7fe8fff8..36f53ba11 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java @@ -38,13 +38,13 @@ public interface HoodieRecordPayload extends Seri /** * This methods lets you write custom merging/combining logic to produce new values as a function * of current value on storage and whats contained in this object. - * + *

* eg: 1) You are updating counters, you may want to add counts to currentValue and write back * updated counts 2) You may be reading DB redo logs, and merge them with current image for a * database row on storage * * @param currentValue Current value in storage, to merge/combine this payload with - * @param schema Schema used for record + * @param schema Schema used for record * @return new combined/merged value to be written back to storage. EMPTY to skip writing this * record. */ diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java index 00564d627..b9bb92544 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java @@ -18,16 +18,16 @@ package com.uber.hoodie.common.model; /** * Type of the Hoodie Table. - * + *

* Currently, 1 type is supported - * + *

* COPY_ON_WRITE - Performs upserts by versioning entire files, with later versions containing newer * value of a record. - * + *

* In the future, following might be added. - * + *

* MERGE_ON_READ - Speeds up upserts, by delaying merge until enough work piles up. - * + *

* SIMPLE_LSM - A simple 2 level LSM tree. */ public enum HoodieTableType { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java index dccfd3105..4706e25a3 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java @@ -17,9 +17,8 @@ package com.uber.hoodie.common.model; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; - -import javax.annotation.Nullable; import java.io.Serializable; +import javax.annotation.Nullable; import org.apache.hadoop.fs.Path; /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java b/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java index dc9e04043..751f73c50 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/storage/SizeAwareDataInputStream.java @@ -25,39 +25,39 @@ import java.util.concurrent.atomic.AtomicInteger; */ public class SizeAwareDataInputStream { - private final DataInputStream dis; - private final AtomicInteger numberOfBytesRead; + private final DataInputStream dis; + private final AtomicInteger numberOfBytesRead; - public SizeAwareDataInputStream(DataInputStream dis) { - this.dis = dis; - this.numberOfBytesRead = new AtomicInteger(0); - } + public SizeAwareDataInputStream(DataInputStream dis) { + this.dis = dis; + this.numberOfBytesRead = new AtomicInteger(0); + } - public int readInt() throws IOException { - numberOfBytesRead.addAndGet(Integer.BYTES); - return dis.readInt(); - } + public int readInt() throws IOException { + numberOfBytesRead.addAndGet(Integer.BYTES); + return dis.readInt(); + } - public void readFully(byte b[], int off, int len) throws IOException { - numberOfBytesRead.addAndGet(len); - dis.readFully(b, off, len); - } + public void readFully(byte[] b, int off, int len) throws IOException { + numberOfBytesRead.addAndGet(len); + dis.readFully(b, off, len); + } - public void readFully(byte b[]) throws IOException { - numberOfBytesRead.addAndGet(b.length); - dis.readFully(b); - } + public void readFully(byte[] b) throws IOException { + numberOfBytesRead.addAndGet(b.length); + dis.readFully(b); + } - public int skipBytes(int n) throws IOException { - numberOfBytesRead.addAndGet(n); - return dis.skipBytes(n); - } + public int skipBytes(int n) throws IOException { + numberOfBytesRead.addAndGet(n); + return dis.skipBytes(n); + } - public void close() throws IOException { - dis.close(); - } + public void close() throws IOException { + dis.close(); + } - public Integer getNumberOfBytesRead() { - return numberOfBytesRead.get(); - } + public Integer getNumberOfBytesRead() { + return numberOfBytesRead.get(); + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java index 577d7cf4e..56921fb0c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java @@ -42,7 +42,7 @@ import org.apache.log4j.Logger; */ public class HoodieTableConfig implements Serializable { - private final transient static Logger log = LogManager.getLogger(HoodieTableConfig.class); + private static final transient Logger log = LogManager.getLogger(HoodieTableConfig.class); public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties"; public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name"; 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 1388823af..9462c301d 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 @@ -48,7 +48,7 @@ import org.apache.log4j.Logger; */ public class HoodieTableMetaClient implements Serializable { - private final transient static Logger log = LogManager.getLogger(HoodieTableMetaClient.class); + private static final transient Logger log = LogManager.getLogger(HoodieTableMetaClient.class); public static String METAFOLDER_NAME = ".hoodie"; public static String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp"; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index 5dcb2c83c..7cd4ee307 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -16,6 +16,9 @@ package com.uber.hoodie.common.table.log; +import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; +import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; + import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; @@ -28,17 +31,9 @@ import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.SpillableMapUtils; import com.uber.hoodie.common.util.collection.ExternalSpillableMap; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; +import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.exception.HoodieIOException; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; @@ -49,30 +44,28 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; - -import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; -import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** - * Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of - * records which will be used as a lookup table when merging the base columnar file with the redo - * log file. - * NOTE: If readBlockLazily is turned on, does not merge, instead keeps reading log blocks and merges everything at once - * This is an optimization to avoid seek() back and forth to read new block (forward seek()) - * and lazily read content of seen block (reverse and forward seek()) during merge - * | | Read Block 1 Metadata | | Read Block 1 Data | - * | | Read Block 2 Metadata | | Read Block 2 Data | - * | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | - * | | Read Block N Metadata | | Read Block N Data | - * - * This results in two I/O passes over the log file. - * + * Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of records which will + * be used as a lookup table when merging the base columnar file with the redo log file. NOTE: If readBlockLazily is + * turned on, does not merge, instead keeps reading log blocks and merges everything at once This is an optimization to + * avoid seek() back and forth to read new block (forward seek()) and lazily read content of seen block (reverse and + * forward seek()) during merge | | Read Block 1 Metadata | | Read Block 1 Data | | | Read Block 2 + * Metadata | | Read Block 2 Data | | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | | + * | Read Block N Metadata | | Read Block N Data |

This results in two I/O passes over the log file. */ public class HoodieCompactedLogRecordScanner implements Iterable> { - private final static Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class); + private static final Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class); // Final map of compacted/merged records private final ExternalSpillableMap> records; @@ -116,10 +109,10 @@ public class HoodieCompactedLogRecordScanner implements totalLogFiles.incrementAndGet(); // Use the HoodieLogFileReader to iterate through the blocks in the log file HoodieLogBlock r = logFormatReaderWrapper.next(); - if (r.getBlockType() != CORRUPT_BLOCK && - !HoodieTimeline.compareTimestamps(r.getLogBlockHeader().get(INSTANT_TIME), - this.latestInstantTime, - HoodieTimeline.LESSER_OR_EQUAL)) { + if (r.getBlockType() != CORRUPT_BLOCK + && !HoodieTimeline.compareTimestamps(r.getLogBlockHeader().get(INSTANT_TIME), + this.latestInstantTime, + HoodieTimeline.LESSER_OR_EQUAL)) { //hit a block with instant time greater than should be processed, stop processing further break; } @@ -147,14 +140,16 @@ public class HoodieCompactedLogRecordScanner implements case COMMAND_BLOCK: // Consider the following scenario // (Time 0, C1, Task T1) -> Running - // (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct DataBlock (B1) with commitTime C1 + // (Time 1, C1, Task T1) -> Failed (Wrote either a corrupt block or a correct + // DataBlock (B1) with commitTime C1 // (Time 2, C1, Task T1.2) -> Running (Task T1 was retried and the attempt number is 2) // (Time 3, C1, Task T1.2) -> Finished (Wrote a correct DataBlock B2) // Now a logFile L1 can have 2 correct Datablocks (B1 and B2) which are the same. // Say, commit C1 eventually failed and a rollback is triggered. - // Rollback will write only 1 rollback block (R1) since it assumes one block is written per ingestion batch for a file, - // but in reality we need to rollback (B1 & B2) - // The following code ensures the same rollback block (R1) is used to rollback both B1 & B2 + // Rollback will write only 1 rollback block (R1) since it assumes one block is + // written per ingestion batch for a file but in reality we need to rollback (B1 & B2) + // The following code ensures the same rollback block (R1) is used to rollback + // both B1 & B2 log.info("Reading a command block from file " + logFile.getPath()); // This is a command block - take appropriate action based on the command HoodieCommandBlock commandBlock = (HoodieCommandBlock) r; @@ -163,10 +158,11 @@ public class HoodieCompactedLogRecordScanner implements switch (commandBlock.getType()) { // there can be different types of command blocks case ROLLBACK_PREVIOUS_BLOCK: // Rollback the last read log block - // Get commit time from last record block, compare with targetCommitTime, rollback only if equal, - // this is required in scenarios of invalid/extra rollback blocks written due to failures during - // the rollback operation itself and ensures the same rollback block (R1) is used to rollback - // both B1 & B2 with same instant_time + // Get commit time from last record block, compare with targetCommitTime, + // rollback only if equal, this is required in scenarios of invalid/extra + // rollback blocks written due to failures during the rollback operation itself + // and ensures the same rollback block (R1) is used to rollback both B1 & B2 with + // same instant_time int numBlocksRolledBack = 0; while (!currentInstantLogBlocks.isEmpty()) { HoodieLogBlock lastBlock = currentInstantLogBlocks.peek(); @@ -176,30 +172,29 @@ public class HoodieCompactedLogRecordScanner implements "Rolling back the last corrupted log block read in " + logFile.getPath()); currentInstantLogBlocks.pop(); numBlocksRolledBack++; - } - // rollback last data block or delete block - else if (lastBlock.getBlockType() != CORRUPT_BLOCK && - targetInstantForCommandBlock - .contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) { + } else if (lastBlock.getBlockType() != CORRUPT_BLOCK + && targetInstantForCommandBlock + .contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) { + // rollback last data block or delete block log.info("Rolling back the last log block read in " + logFile.getPath()); currentInstantLogBlocks.pop(); numBlocksRolledBack++; - } - // invalid or extra rollback block - else if (!targetInstantForCommandBlock + } else if (!targetInstantForCommandBlock .contentEquals( currentInstantLogBlocks.peek().getLogBlockHeader().get(INSTANT_TIME))) { - log.warn("TargetInstantTime " + targetInstantForCommandBlock + - " invalid or extra rollback command block in " + logFile.getPath()); + // invalid or extra rollback block + log.warn("TargetInstantTime " + targetInstantForCommandBlock + + " invalid or extra rollback command block in " + logFile.getPath()); break; - } - // this should not happen ideally - else { + } else { + // this should not happen ideally log.warn("Unable to apply rollback command block in " + logFile.getPath()); } } log.info("Number of applied rollback blocks " + numBlocksRolledBack); break; + default: + throw new UnsupportedOperationException("Command type not yet supported."); } break; @@ -208,6 +203,8 @@ public class HoodieCompactedLogRecordScanner implements // If there is a corrupt block - we will assume that this was the next data block currentInstantLogBlocks.push(r); break; + default: + throw new UnsupportedOperationException("Block type not supported yet"); } } // merge the last read block when all the blocks are done reading @@ -240,10 +237,9 @@ public class HoodieCompactedLogRecordScanner implements } /** - * Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge - * with the application specific payload if the same key was found before. Sufficient to just merge - * the log records since the base data is merged on previous compaction. - * Finally, merge this log block with the accumulated records + * Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge with the application + * specific payload if the same key was found before. Sufficient to just merge the log records since the base data is + * merged on previous compaction. Finally, merge this log block with the accumulated records */ private Map> merge( HoodieAvroDataBlock dataBlock) throws IOException { @@ -291,6 +287,9 @@ public class HoodieCompactedLogRecordScanner implements case CORRUPT_BLOCK: log.warn("Found a corrupt block which was not rolled back"); break; + default: + //TODO : Need to understand if COMMAND_BLOCK has to be handled? + break; } } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java index 1d4cda061..1ccf43cfa 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java @@ -28,18 +28,17 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; import com.uber.hoodie.exception.CorruptedLogFileException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieNotSupportedException; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.EOFException; import java.io.IOException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** * Scans a log file and provides block level iterator on the log file Loads the entire block @@ -49,28 +48,29 @@ import java.util.Optional; class HoodieLogFileReader implements HoodieLogFormat.Reader { private static final int DEFAULT_BUFFER_SIZE = 4096; - private final static Logger log = LogManager.getLogger(HoodieLogFileReader.class); + private static final Logger log = LogManager.getLogger(HoodieLogFileReader.class); private final FSDataInputStream inputStream; private final HoodieLogFile logFile; private static final byte[] oldMagicBuffer = new byte[4]; private static final byte[] magicBuffer = new byte[6]; private final Schema readerSchema; - private LogFormatVersion nextBlockVersion; + private HoodieLogFormat.LogFormatVersion nextBlockVersion; private boolean readBlockLazily; private long reverseLogFilePosition; private long lastReverseLogFilePosition; private boolean reverseReader; HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, - boolean readBlockLazily, boolean reverseReader) throws IOException { + boolean readBlockLazily, boolean reverseReader) throws IOException { this.inputStream = fs.open(logFile.getPath(), bufferSize); this.logFile = logFile; this.readerSchema = readerSchema; this.readBlockLazily = readBlockLazily; this.reverseReader = reverseReader; - if(this.reverseReader) { - this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs.getFileStatus(logFile.getPath()).getLen(); + if (this.reverseReader) { + this.reverseLogFilePosition = this.lastReverseLogFilePosition = fs + .getFileStatus(logFile.getPath()).getLen(); } addShutDownHook(); } @@ -80,7 +80,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, readBlockLazily, reverseReader); } - HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) throws IOException { + HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema) + throws IOException { this(fs, logFile, readerSchema, DEFAULT_BUFFER_SIZE, false, false); } @@ -105,7 +106,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { }); } - // TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows for max of Integer size + // TODO : convert content and block length to long by using ByteBuffer, raw byte [] allows + // for max of Integer size private HoodieLogBlock readBlock() throws IOException { int blocksize = -1; @@ -137,8 +139,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { } // We may have had a crash which could have written this block partially - // Skip blocksize in the stream and we should either find a sync marker (start of the next block) or EOF - // If we did not find either of it, then this block is a corrupted block. + // Skip blocksize in the stream and we should either find a sync marker (start of the next + // block) or EOF. If we did not find either of it, then this block is a corrupted block. boolean isCorrupted = isBlockCorrupt(blocksize); if (isCorrupted) { return createCorruptBlock(); @@ -168,7 +170,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { } // 6. Read the content or skip content based on IO vs Memory trade-off by client - // TODO - have a max block size and reuse this buffer in the ByteBuffer (hard to guess max block size for now) + // TODO - have a max block size and reuse this buffer in the ByteBuffer + // (hard to guess max block size for now) long contentPosition = inputStream.getPos(); byte[] content = HoodieLogBlock.readOrSkipContent(inputStream, contentLength, readBlockLazily); @@ -178,7 +181,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { footer = HoodieLogBlock.getLogMetadata(inputStream); } - // 8. Read log block length, if present. This acts as a reverse pointer when traversing a log file in reverse + // 8. Read log block length, if present. This acts as a reverse pointer when traversing a + // log file in reverse long logBlockLength = 0; if (nextBlockVersion.hasLogBlockLength()) { logBlockLength = inputStream.readLong(); @@ -193,15 +197,18 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) { return HoodieAvroDataBlock.getBlock(content, readerSchema); } else { - return HoodieAvroDataBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, - contentPosition, contentLength, blockEndPos, readerSchema, header, footer); + return HoodieAvroDataBlock + .getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, + contentPosition, contentLength, blockEndPos, readerSchema, header, footer); } case DELETE_BLOCK: - return HoodieDeleteBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, - contentPosition, contentLength, blockEndPos, header, footer); + return HoodieDeleteBlock + .getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, + contentPosition, contentLength, blockEndPos, header, footer); case COMMAND_BLOCK: - return HoodieCommandBlock.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, - contentPosition, contentLength, blockEndPos, header, footer); + return HoodieCommandBlock + .getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily, + contentPosition, contentLength, blockEndPos, header, footer); default: throw new HoodieNotSupportedException("Unsupported Block " + blockType); } @@ -216,9 +223,12 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { log.info("Next available block in " + logFile + " starts at " + nextBlockOffset); int corruptedBlockSize = (int) (nextBlockOffset - currentPos); long contentPosition = inputStream.getPos(); - byte[] corruptedBytes = HoodieLogBlock.readOrSkipContent(inputStream, corruptedBlockSize, readBlockLazily); - return HoodieCorruptBlock.getBlock(logFile, inputStream, Optional.ofNullable(corruptedBytes), readBlockLazily, - contentPosition, corruptedBlockSize, corruptedBlockSize, new HashMap<>(), new HashMap<>()); + byte[] corruptedBytes = HoodieLogBlock + .readOrSkipContent(inputStream, corruptedBlockSize, readBlockLazily); + return HoodieCorruptBlock + .getBlock(logFile, inputStream, Optional.ofNullable(corruptedBytes), readBlockLazily, + contentPosition, corruptedBlockSize, corruptedBlockSize, new HashMap<>(), + new HashMap<>()); } private boolean isBlockCorrupt(int blocksize) throws IOException { @@ -278,12 +288,10 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { } /** - * Read log format version from log file, if present - * For old log files written with Magic header OLD_MAGIC and without version, return DEFAULT_VERSION - * - * @throws IOException + * Read log format version from log file, if present For old log files written with Magic header + * OLD_MAGIC and without version, return DEFAULT_VERSION */ - private LogFormatVersion readVersion() throws IOException { + private HoodieLogFormat.LogFormatVersion readVersion() throws IOException { // If not old log file format (written with Magic header OLD_MAGIC), then read log version if (Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) { Arrays.fill(oldMagicBuffer, (byte) 0); @@ -309,7 +317,8 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { inputStream.readFully(oldMagicBuffer, 0, 4); if (!Arrays.equals(oldMagicBuffer, HoodieLogFormat.OLD_MAGIC)) { throw new CorruptedLogFileException( - logFile + "could not be read. Did not find the magic bytes at the start of the block"); + logFile + + "could not be read. Did not find the magic bytes at the start of the block"); } } return false; @@ -324,19 +333,17 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { try { // hasNext() must be called before next() return readBlock(); - } catch(IOException io) { + } catch (IOException io) { throw new HoodieIOException("IOException when reading logblock from log file " + logFile, io); } } /** * hasPrev is not idempotent - * - * @return */ public boolean hasPrev() { try { - if(!this.reverseReader) { + if (!this.reverseReader) { throw new HoodieNotSupportedException("Reverse log reader has not been enabled"); } reverseLogFilePosition = lastReverseLogFilePosition; @@ -351,17 +358,13 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { } /** - * This is a reverse iterator - * Note: At any point, an instance of HoodieLogFileReader should either iterate reverse (prev) - * or forward (next). Doing both in the same instance is not supported + * This is a reverse iterator Note: At any point, an instance of HoodieLogFileReader should either + * iterate reverse (prev) or forward (next). Doing both in the same instance is not supported * WARNING : Every call to prev() should be preceded with hasPrev() - * - * @return - * @throws IOException */ public HoodieLogBlock prev() throws IOException { - if(!this.reverseReader) { + if (!this.reverseReader) { throw new HoodieNotSupportedException("Reverse log reader has not been enabled"); } long blockSize = inputStream.readLong(); @@ -372,8 +375,9 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { } catch (Exception e) { // this could be a corrupt block inputStream.seek(blockEndPos); - throw new CorruptedLogFileException("Found possible corrupted block, cannot read log file in reverse, " + - "fallback to forward reading of logfile"); + throw new CorruptedLogFileException( + "Found possible corrupted block, cannot read log file in reverse, " + + "fallback to forward reading of logfile"); } boolean hasNext = hasNext(); reverseLogFilePosition -= blockSize; @@ -382,16 +386,14 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader { } /** - * Reverse pointer, does not read the block. Return the current position of the log file (in reverse) - * If the pointer (inputstream) is moved in any way, it is the job of the client of this class to - * seek/reset it back to the file position returned from the method to expect correct results - * - * @return - * @throws IOException + * Reverse pointer, does not read the block. Return the current position of the log file (in + * reverse) If the pointer (inputstream) is moved in any way, it is the job of the client of this + * class to seek/reset it back to the file position returned from the method to expect correct + * results */ public long moveToPrev() throws IOException { - if(!this.reverseReader) { + if (!this.reverseReader) { throw new HoodieNotSupportedException("Reverse log reader has not been enabled"); } inputStream.seek(lastReverseLogFilePosition); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java index d5deb9d03..cb12f9ef8 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java @@ -19,22 +19,21 @@ package com.uber.hoodie.common.table.log; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.FSUtils; +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.io.Closeable; -import java.io.IOException; -import java.util.Iterator; - /** - * File Format for Hoodie Log Files. The File Format consists of blocks each seperated with a OLD_MAGIC - * sync marker. A Block can either be a Data block, Command block or Delete Block. Data Block - - * Contains log records serialized as Avro Binary Format Command Block - Specific commands like - * RoLLBACK_PREVIOUS-BLOCK - Tombstone for the previously written block Delete Block - List of keys - * to delete - tombstone for keys + * File Format for Hoodie Log Files. The File Format consists of blocks each seperated with a + * OLD_MAGIC sync marker. A Block can either be a Data block, Command block or Delete Block. Data + * Block - Contains log records serialized as Avro Binary Format Command Block - Specific commands + * like RoLLBACK_PREVIOUS-BLOCK - Tombstone for the previously written block Delete Block - List of + * keys to delete - tombstone for keys */ public interface HoodieLogFormat { @@ -43,19 +42,18 @@ public interface HoodieLogFormat { * this file specific (generate a random 4 byte magic and stick it in the file header), but this I * think is suffice for now - PR */ - byte[] OLD_MAGIC = new byte[]{'H', 'U', 'D', 'I'}; + byte[] OLD_MAGIC = new byte[] {'H', 'U', 'D', 'I'}; /** - * Magic 6 bytes we put at the start of every block in the log file. - * This is added to maintain backwards compatiblity due to lack of log format/block - * version in older log files. All new log block will now write this OLD_MAGIC value + * Magic 6 bytes we put at the start of every block in the log file. This is added to maintain + * backwards compatiblity due to lack of log format/block version in older log files. All new log + * block will now write this OLD_MAGIC value */ - byte[] MAGIC = new byte[]{'#', 'H', 'U', 'D', 'I', '#'}; + byte[] MAGIC = new byte[] {'#', 'H', 'U', 'D', 'I', '#'}; /** - * The current version of the log format. Anytime the log format changes - * this version needs to be bumped and corresponding changes need to be made to - * {@link HoodieLogFormatVersion} + * The current version of the log format. Anytime the log format changes this version needs to be + * bumped and corresponding changes need to be made to {@link HoodieLogFormatVersion} */ int currentVersion = 1; @@ -94,7 +92,7 @@ public interface HoodieLogFormat { */ class WriterBuilder { - private final static Logger log = LogManager.getLogger(WriterBuilder.class); + private static final Logger log = LogManager.getLogger(WriterBuilder.class); // Default max log file size 512 MB public static final long DEFAULT_SIZE_THRESHOLD = 512 * 1024 * 1024L; @@ -112,7 +110,8 @@ public interface HoodieLogFormat { private String logFileId; // File Commit Time stamp private String commitTime; - // version number for this log file. If not specified, then the current version will be computed by inspecting the file system + // version number for this log file. If not specified, then the current version will be + // computed by inspecting the file system private Integer logVersion; // Location of the directory containing the log private Path parentPath; @@ -215,4 +214,35 @@ public interface HoodieLogFormat { throws IOException { return new HoodieLogFileReader(fs, logFile, readerSchema, false, false); } + + /** + * A set of feature flags associated with a log format. Versions are changed when the log format + * changes. TODO(na) - Implement policies around major/minor versions + */ + abstract class LogFormatVersion { + + private final int version; + + LogFormatVersion(int version) { + this.version = version; + } + + public int getVersion() { + return version; + } + + public abstract boolean hasMagicHeader(); + + public abstract boolean hasContent(); + + public abstract boolean hasContentLength(); + + public abstract boolean hasOrdinal(); + + public abstract boolean hasHeader(); + + public abstract boolean hasFooter(); + + public abstract boolean hasLogBlockLength(); + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java index e5f8f5c99..00a3a7498 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatReader.java @@ -19,11 +19,10 @@ package com.uber.hoodie.common.table.log; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.exception.HoodieIOException; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; - import java.io.IOException; import java.util.List; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -36,16 +35,16 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { private final boolean readBlocksLazily; private final boolean reverseLogReader; - private final static Logger log = LogManager.getLogger(HoodieLogFormatReader.class); + private static final Logger log = LogManager.getLogger(HoodieLogFormatReader.class); HoodieLogFormatReader(FileSystem fs, List logFiles, - Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException { + Schema readerSchema, boolean readBlocksLazily, boolean reverseLogReader) throws IOException { this.logFiles = logFiles; this.fs = fs; this.readerSchema = readerSchema; this.readBlocksLazily = readBlocksLazily; this.reverseLogReader = reverseLogReader; - if(logFiles.size() > 0) { + if (logFiles.size() > 0) { HoodieLogFile nextLogFile = logFiles.remove(0); this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily, false); @@ -53,7 +52,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { } HoodieLogFormatReader(FileSystem fs, List logFiles, - Schema readerSchema) throws IOException { + Schema readerSchema) throws IOException { this(fs, logFiles, readerSchema, false, false); } @@ -67,16 +66,15 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader { @Override public boolean hasNext() { - if(currentReader == null) { + if (currentReader == null) { return false; - } - else if (currentReader.hasNext()) { + } else if (currentReader.hasNext()) { return true; - } - else if (logFiles.size() > 0) { + } else if (logFiles.size() > 0) { try { HoodieLogFile nextLogFile = logFiles.remove(0); - this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, readBlocksLazily, + this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, + readBlocksLazily, false); } catch (IOException io) { throw new HoodieIOException("unable to initialize read with log file ", io); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatVersion.java similarity index 68% rename from hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java rename to hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatVersion.java index 8bba078c7..dc0c75ca5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/LogFormatVersion.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatVersion.java @@ -17,46 +17,17 @@ package com.uber.hoodie.common.table.log; /** - * A set of feature flags associated with a log format. - * Versions are changed when the log format changes. - * TODO(na) - Implement policies around major/minor versions + * Implements logic to determine behavior for feature flags for + * {@link HoodieLogFormat.LogFormatVersion}. */ -abstract class LogFormatVersion { - private final int version; +final class HoodieLogFormatVersion extends HoodieLogFormat.LogFormatVersion { - LogFormatVersion(int version) { - this.version = version; - } - - public int getVersion() { - return version; - } - - public abstract boolean hasMagicHeader(); - - public abstract boolean hasContent(); - - public abstract boolean hasContentLength(); - - public abstract boolean hasOrdinal(); - - public abstract boolean hasHeader(); - - public abstract boolean hasFooter(); - - public abstract boolean hasLogBlockLength(); -} - -/** - * Implements logic to determine behavior for feature flags for {@link LogFormatVersion} - */ -final class HoodieLogFormatVersion extends LogFormatVersion { - - public final static int DEFAULT_VERSION = 0; + public static final int DEFAULT_VERSION = 0; HoodieLogFormatVersion(int version) { super(version); } + @Override public boolean hasMagicHeader() { switch (super.getVersion()) { @@ -114,8 +85,9 @@ final class HoodieLogFormatVersion extends LogFormatVersion { return false; case 1: return true; + default: + return false; } - return false; } @Override @@ -125,7 +97,8 @@ final class HoodieLogFormatVersion extends LogFormatVersion { return false; case 1: return true; + default: + return false; } - return false; } } \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java index 9ea4600a5..ee1dcf559 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormatWriter.java @@ -22,6 +22,7 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat.WriterBuilder; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieException; +import java.io.IOException; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -31,15 +32,13 @@ import org.apache.hadoop.ipc.RemoteException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.io.IOException; - /** * HoodieLogFormatWriter can be used to append blocks to a log file Use * HoodieLogFormat.WriterBuilder to construct */ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { - private final static Logger log = LogManager.getLogger(HoodieLogFormatWriter.class); + private static final Logger log = LogManager.getLogger(HoodieLogFormatWriter.class); private HoodieLogFile logFile; private final FileSystem fs; @@ -49,7 +48,6 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { private FSDataOutputStream output; /** - * * @param fs * @param logFile * @param bufferSize @@ -71,7 +69,8 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { try { this.output = fs.append(path, bufferSize); } catch (RemoteException e) { - // this happens when either another task executor writing to this file died or data node is going down + // this happens when either another task executor writing to this file died or + // data node is going down if (e.getClassName().equals(AlreadyBeingCreatedException.class.getName()) && fs instanceof DistributedFileSystem) { log.warn("Trying to recover log on path " + path); @@ -120,21 +119,23 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { throws IOException, InterruptedException { // Find current version - LogFormatVersion currentLogFormatVersion = new HoodieLogFormatVersion(HoodieLogFormat.currentVersion); + HoodieLogFormat.LogFormatVersion currentLogFormatVersion = new HoodieLogFormatVersion( + HoodieLogFormat.currentVersion); long currentSize = this.output.size(); // 1. Write the magic header for the start of the block this.output.write(HoodieLogFormat.MAGIC); // bytes for header - byte [] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader()); + byte[] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader()); // content bytes - byte [] content = block.getContentBytes(); + byte[] content = block.getContentBytes(); // bytes for footer - byte [] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter()); + byte[] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter()); // 2. Write the total size of the block (excluding Magic) - this.output.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length)); + this.output + .writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length)); // 3. Write the version of this log block this.output.writeInt(currentLogFormatVersion.getVersion()); @@ -149,7 +150,8 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { this.output.write(content); // 8. Write the footers for the log block this.output.write(footerBytes); - // 9. Write the total size of the log block (including magic) which is everything written until now (for reverse pointer) + // 9. Write the total size of the log block (including magic) which is everything written + // until now (for reverse pointer) this.output.writeLong(this.output.size() - currentSize); // Flush every block to disk flush(); @@ -159,35 +161,27 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { } /** - * - * This method returns the total LogBlock Length which is the sum of - * 1. Number of bytes to write version - * 2. Number of bytes to write ordinal - * 3. Length of the headers - * 4. Number of bytes used to write content length - * 5. Length of the content - * 6. Length of the footers - * 7. Number of bytes to write totalLogBlockLength - * @param contentLength - * @param headerLength - * @param footerLength - * @return + * This method returns the total LogBlock Length which is the sum of 1. Number of bytes to write + * version 2. Number of bytes to write ordinal 3. Length of the headers 4. Number of bytes used to + * write content length 5. Length of the content 6. Length of the footers 7. Number of bytes to + * write totalLogBlockLength */ private int getLogBlockLength(int contentLength, int headerLength, int footerLength) { return Integer.BYTES + // Number of bytes to write version - Integer.BYTES + // Number of bytes to write ordinal - headerLength + // Length of the headers - Long.BYTES + // Number of bytes used to write content length - contentLength + // Length of the content - footerLength + // Length of the footers - Long.BYTES; // Number of bytes to write totalLogBlockLength at end of block (for reverse pointer) + Integer.BYTES + // Number of bytes to write ordinal + headerLength + // Length of the headers + Long.BYTES + // Number of bytes used to write content length + contentLength + // Length of the content + footerLength + // Length of the footers + Long.BYTES; // bytes to write totalLogBlockLength at end of block (for reverse ptr) } private Writer rolloverIfNeeded() throws IOException, InterruptedException { // Roll over if the size is past the threshold if (getCurrentSize() > sizeThreshold) { - //TODO - make an end marker which seals the old log file (no more appends possible to that file). + //TODO - make an end marker which seals the old log file (no more appends possible to that + // file). log.info("CurrentSize " + getCurrentSize() + " has reached threshold " + sizeThreshold + ". Rolling over to the next version"); HoodieLogFile newLogFile = logFile.rollOver(fs); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java index eb2b5f45f..b8822521f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java @@ -44,12 +44,9 @@ import org.apache.avro.io.EncoderFactory; import org.apache.hadoop.fs.FSDataInputStream; /** - * DataBlock contains a list of records serialized using Avro. - * The Datablock contains - * 1. Data Block version - * 2. Total number of records in the block - * 3. Size of a record - * 4. Actual avro serialized content of the record + * DataBlock contains a list of records serialized using Avro. The Datablock contains 1. Data Block + * version 2. Total number of records in the block 3. Size of a record 4. Actual avro serialized + * content of the record */ public class HoodieAvroDataBlock extends HoodieLogBlock { @@ -216,7 +213,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock { deflate(); } - /*****************************************************DEPRECATED METHODS**********************************************/ + /*********************************DEPRECATED METHODS***********************************/ @Deprecated @VisibleForTesting @@ -232,7 +229,8 @@ public class HoodieAvroDataBlock extends HoodieLogBlock { @Deprecated /** - * This method is retained to provide backwards compatibility to HoodieArchivedLogs which were written using HoodieLogFormat V1 + * This method is retained to provide backwards compatibility to HoodieArchivedLogs which + * were written using HoodieLogFormat V1 */ public static HoodieLogBlock getBlock(byte[] content, Schema readerSchema) throws IOException { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlockVersion.java new file mode 100644 index 000000000..796d364e5 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlockVersion.java @@ -0,0 +1,37 @@ +/* + * 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.common.table.log.block; + +/** + * A set of feature flags associated with a data log block format. Versions are changed when the log + * block format changes. TODO(na) - Implement policies around major/minor versions + */ +final class HoodieAvroDataBlockVersion extends HoodieLogBlockVersion { + + HoodieAvroDataBlockVersion(int version) { + super(version); + } + + public boolean hasRecordCount() { + switch (super.getVersion()) { + case DEFAULT_VERSION: + return true; + default: + return true; + } + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java index b9b11cc8f..ba520f9ce 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java @@ -17,11 +17,10 @@ package com.uber.hoodie.common.table.log.block; import com.uber.hoodie.common.model.HoodieLogFile; -import org.apache.hadoop.fs.FSDataInputStream; - import java.util.HashMap; import java.util.Map; import java.util.Optional; +import org.apache.hadoop.fs.FSDataInputStream; /** * Command block issues a specific command to the scanner @@ -30,7 +29,9 @@ public class HoodieCommandBlock extends HoodieLogBlock { private final HoodieCommandBlockTypeEnum type; - public enum HoodieCommandBlockTypeEnum {ROLLBACK_PREVIOUS_BLOCK} + public enum HoodieCommandBlockTypeEnum { + ROLLBACK_PREVIOUS_BLOCK + } public HoodieCommandBlock(Map header) { this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>()); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlockVersion.java new file mode 100644 index 000000000..96a56f572 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlockVersion.java @@ -0,0 +1,28 @@ +/* + * 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.common.table.log.block; + +/** + * A set of feature flags associated with a command log block format. Versions are changed when the + * log block format changes. TODO(na) - Implement policies around major/minor versions + */ +final class HoodieCommandBlockVersion extends HoodieLogBlockVersion { + + HoodieCommandBlockVersion(int version) { + super(version); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java index c75c8ea62..fc0402f08 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java @@ -17,11 +17,10 @@ package com.uber.hoodie.common.table.log.block; import com.uber.hoodie.common.model.HoodieLogFile; -import org.apache.hadoop.fs.FSDataInputStream; - import java.io.IOException; import java.util.Map; import java.util.Optional; +import org.apache.hadoop.fs.FSDataInputStream; /** * Corrupt block is emitted whenever the scanner finds the length of the block written at the @@ -61,6 +60,7 @@ public class HoodieCorruptBlock extends HoodieLogBlock { Map footer) throws IOException { return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily, - Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer); + Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), + header, footer); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java index 4de25b5da..2f4ed7421 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java @@ -19,9 +19,6 @@ package com.uber.hoodie.common.table.log.block; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.storage.SizeAwareDataInputStream; import com.uber.hoodie.exception.HoodieIOException; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.FSDataInputStream; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -31,6 +28,8 @@ import java.nio.charset.Charset; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FSDataInputStream; /** * Delete block contains a list of keys to be deleted from scanning the blocks so far @@ -65,7 +64,8 @@ public class HoodieDeleteBlock extends HoodieLogBlock { ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream output = new DataOutputStream(baos); - byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ',').getBytes(Charset.forName("utf-8")); + byte[] bytesToWrite = StringUtils.join(getKeysToDelete(), ',') + .getBytes(Charset.forName("utf-8")); output.writeInt(HoodieLogBlock.version); output.writeInt(bytesToWrite.length); output.write(bytesToWrite); @@ -80,7 +80,8 @@ public class HoodieDeleteBlock extends HoodieLogBlock { inflate(); } SizeAwareDataInputStream dis = - new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(getContent().get()))); + new SizeAwareDataInputStream( + new DataInputStream(new ByteArrayInputStream(getContent().get()))); int version = dis.readInt(); int dataLength = dis.readInt(); byte[] data = new byte[dataLength]; @@ -110,6 +111,7 @@ public class HoodieDeleteBlock extends HoodieLogBlock { Map footer) throws IOException { return new HoodieDeleteBlock(content, inputStream, readBlockLazily, - Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), header, footer); + Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)), + header, footer); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlockVersion.java new file mode 100644 index 000000000..990af81ca --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlockVersion.java @@ -0,0 +1,28 @@ +/* + * 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.common.table.log.block; + +/** + * A set of feature flags associated with a delete log block format. Versions are changed when the + * log block format changes. TODO(na) - Implement policies around major/minor versions + */ +final class HoodieDeleteBlockVersion extends HoodieLogBlockVersion { + + HoodieDeleteBlockVersion(int version) { + super(version); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java index e7735b0db..86b3f2698 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java @@ -36,10 +36,10 @@ import org.apache.hadoop.fs.FSDataInputStream; public abstract class HoodieLogBlock { /** - * The current version of the log block. Anytime the logBlock format changes - * this version needs to be bumped and corresponding changes need to be made to - * {@link HoodieLogBlockVersion} - * TODO : Change this to a class, something like HoodieLogBlockVersionV1/V2 and implement/override operations there + * The current version of the log block. Anytime the logBlock format changes this version needs to + * be bumped and corresponding changes need to be made to {@link HoodieLogBlockVersion} TODO : + * Change this to a class, something like HoodieLogBlockVersionV1/V2 and implement/override + * operations there */ public static int version = 1; // Header for each log block @@ -49,7 +49,7 @@ public abstract class HoodieLogBlock { // Location of a log block on disk private final Optional blockContentLocation; // data for a specific block - private Optional content; + private Optional content; // TODO : change this to just InputStream so this works for any FileSystem // create handlers to return specific type of inputstream based on FS // input stream corresponding to the log file where this logBlock belongs @@ -57,10 +57,10 @@ public abstract class HoodieLogBlock { // Toggle flag, whether to read blocks lazily (I/O intensive) or not (Memory intensive) protected boolean readBlockLazily; - public HoodieLogBlock(@Nonnull Map logBlockHeader, + public HoodieLogBlock(@Nonnull Map logBlockHeader, @Nonnull Map logBlockFooter, @Nonnull Optional blockContentLocation, - @Nonnull Optional content, + @Nonnull Optional content, FSDataInputStream inputStream, boolean readBlockLazily) { this.logBlockHeader = logBlockHeader; @@ -76,7 +76,7 @@ public abstract class HoodieLogBlock { throw new HoodieException("No implementation was provided"); } - public byte [] getMagic() { + public byte[] getMagic() { throw new HoodieException("No implementation was provided"); } @@ -116,8 +116,8 @@ public abstract class HoodieLogBlock { } /** - * Log Metadata headers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal. - * Only add new enums at the end. + * Log Metadata headers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the + * ordinal. Only add new enums at the end. */ public enum HeaderMetadataType { INSTANT_TIME, @@ -127,17 +127,19 @@ public abstract class HoodieLogBlock { } /** - * Log Metadata footers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the ordinal. - * Only add new enums at the end. + * Log Metadata footers abstraction for a HoodieLogBlock WARNING : This enum is serialized as the + * ordinal. Only add new enums at the end. */ public enum FooterMetadataType { } /** - * This class is used to store the Location of the Content of a Log Block. It's used when a client chooses for a - * IO intensive CompactedScanner, the location helps to lazily read contents from the log file + * This class is used to store the Location of the Content of a Log Block. It's used when a client + * chooses for a IO intensive CompactedScanner, the location helps to lazily read contents from + * the log file */ public static final class HoodieLogBlockContentLocation { + // The logFile that contains this block private final HoodieLogFile logFile; // The filePosition in the logFile for the contents of this block @@ -147,7 +149,8 @@ public abstract class HoodieLogBlock { // The final position where the complete block ends private final long blockEndPos; - HoodieLogBlockContentLocation(HoodieLogFile logFile, long contentPositionInLogFile, long blockSize, long blockEndPos) { + HoodieLogBlockContentLocation(HoodieLogFile logFile, long contentPositionInLogFile, + long blockSize, long blockEndPos) { this.logFile = logFile; this.contentPositionInLogFile = contentPositionInLogFile; this.blockSize = blockSize; @@ -190,7 +193,8 @@ public abstract class HoodieLogBlock { } /** - * Convert bytes to LogMetadata, follow the same order as {@link HoodieLogBlock#getLogMetadataBytes} + * Convert bytes to LogMetadata, follow the same order as + * {@link HoodieLogBlock#getLogMetadataBytes} */ public static Map getLogMetadata(DataInputStream dis) throws IOException { @@ -216,16 +220,10 @@ public abstract class HoodieLogBlock { /** * Read or Skip block content of a log block in the log file. Depends on lazy reading enabled in * {@link com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner} - * - * @param inputStream - * @param contentLength - * @param readBlockLazily - * @return - * @throws IOException */ - public static byte [] readOrSkipContent(FSDataInputStream inputStream, + public static byte[] readOrSkipContent(FSDataInputStream inputStream, Integer contentLength, boolean readBlockLazily) throws IOException { - byte [] content = null; + byte[] content = null; if (!readBlockLazily) { // Read the contents in memory content = new byte[contentLength]; @@ -239,7 +237,6 @@ public abstract class HoodieLogBlock { /** * When lazyReading of blocks is turned on, inflate the content of a log block from disk - * @throws IOException */ protected void inflate() throws IOException { @@ -248,21 +245,21 @@ public abstract class HoodieLogBlock { inputStream.seek(this.getBlockContentLocation().get().getContentPositionInLogFile()); inputStream.readFully(content.get(), 0, content.get().length); inputStream.seek(this.getBlockContentLocation().get().getBlockEndPos()); - } catch(IOException e) { + } catch (IOException e) { try { // TODO : fs.open() and return inputstream again, need to pass FS configuration // because the inputstream might close/timeout for large number of log blocks to be merged inflate(); - } catch(IOException io) { + } catch (IOException io) { throw new HoodieIOException("unable to lazily read log block from disk", io); } } } /** - * After the content bytes is converted into the required DataStructure by a logBlock, deflate the content - * to release byte [] and relieve memory pressure when GC kicks in. - * NOTE: This still leaves the heap fragmented + * After the content bytes is converted into the required DataStructure by a logBlock, deflate the + * content to release byte [] and relieve memory pressure when GC kicks in. NOTE: This still + * leaves the heap fragmented */ protected void deflate() { content = Optional.empty(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlockVersion.java new file mode 100644 index 000000000..9316099ca --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlockVersion.java @@ -0,0 +1,33 @@ +/* + * 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.common.table.log.block; + +abstract class HoodieLogBlockVersion { + + private final int currentVersion; + + public static final int DEFAULT_VERSION = 0; + + HoodieLogBlockVersion(int version) { + this.currentVersion = version; + } + + int getVersion() { + return currentVersion; + } +} + diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java deleted file mode 100644 index 1a9844209..000000000 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/LogBlockVersion.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.common.table.log.block; - -import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.version; - -abstract class HoodieLogBlockVersion { - - private final int currentVersion; - - public final static int DEFAULT_VERSION = 0; - - HoodieLogBlockVersion(int version) { - this.currentVersion = version; - } - - int getVersion() { - return currentVersion; - } -} - -/** - * A set of feature flags associated with a data log block format. - * Versions are changed when the log block format changes. - * TODO(na) - Implement policies around major/minor versions - */ -final class HoodieAvroDataBlockVersion extends HoodieLogBlockVersion { - - HoodieAvroDataBlockVersion(int version) { - super(version); - } - - public boolean hasRecordCount() { - switch (super.getVersion()) { - case DEFAULT_VERSION: - return true; - default: - return true; - } - } -} - -/** - * A set of feature flags associated with a command log block format. - * Versions are changed when the log block format changes. - * TODO(na) - Implement policies around major/minor versions - */ -final class HoodieCommandBlockVersion extends HoodieLogBlockVersion { - - HoodieCommandBlockVersion(int version) { - super(version); - } -} - -/** - * A set of feature flags associated with a delete log block format. - * Versions are changed when the log block format changes. - * TODO(na) - Implement policies around major/minor versions - */ -final class HoodieDeleteBlockVersion extends HoodieLogBlockVersion { - - HoodieDeleteBlockVersion(int version) { - super(version); - } -} \ No newline at end of file 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 8c34717b2..a8d35411e 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 @@ -51,9 +51,10 @@ import org.apache.log4j.Logger; */ public class HoodieActiveTimeline extends HoodieDefaultTimeline { - public static final FastDateFormat COMMIT_FORMATTER = FastDateFormat.getInstance("yyyyMMddHHmmss"); + public static final FastDateFormat COMMIT_FORMATTER = FastDateFormat + .getInstance("yyyyMMddHHmmss"); - private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class); + private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class); private HoodieTableMetaClient metaClient; /** @@ -71,12 +72,12 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { Arrays.stream( HoodieTableMetaClient .scanFiles(metaClient.getFs(), new Path(metaClient.getMetaPath()), path -> { - // Include only the meta files with extensions that needs to be included - String extension = FSUtils.getFileExtension(path.getName()); - return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension)); - })).sorted(Comparator.comparing( - // Sort the meta-data by the instant time (first part of the file name) - fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName()))) + // Include only the meta files with extensions that needs to be included + String extension = FSUtils.getFileExtension(path.getName()); + return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension)); + })).sorted(Comparator.comparing( + // Sort the meta-data by the instant time (first part of the file name) + fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName()))) // create HoodieInstantMarkers from FileStatus, which extracts properties .map(HoodieInstant::new).collect(Collectors.toList()); log.info("Loaded instants " + instants); @@ -84,13 +85,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { throw new HoodieIOException("Failed to scan metadata", e); } this.metaClient = metaClient; - // multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 - this.details = (Function> & Serializable) this::getInstantDetails; + // multiple casts will make this lambda serializable - + // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 + this.details = + (Function> & Serializable) this::getInstantDetails; } public HoodieActiveTimeline(HoodieTableMetaClient metaClient) { this(metaClient, - new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, + new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION}); } @@ -114,8 +117,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { } /** - * Get all instants (commits, delta commits) that produce new data, in the active - * timeline * + * Get all instants (commits, delta commits) that produce new data, in the active timeline * */ public HoodieTimeline getCommitsTimeline() { return getTimelineOfActions( @@ -123,8 +125,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { } /** - * Get all instants (commits, delta commits, clean, savepoint, rollback) that result - * in actions, in the active timeline * + * Get all instants (commits, delta commits, clean, savepoint, rollback) that result in actions, + * in the active timeline * */ public HoodieTimeline getAllCommitsTimeline() { return getTimelineOfActions( diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java index 793d9d996..1da29f669 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java @@ -46,7 +46,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline { private HoodieTableMetaClient metaClient; private Map readCommits = new HashMap<>(); - private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class); + private static final transient Logger log = LogManager.getLogger(HoodieArchivedTimeline.class); public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) { // Read back the commits to make sure @@ -68,8 +68,10 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline { throw new HoodieIOException( "Could not load archived commit timeline from path " + archiveLogPath, e); } - // multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 - this.details = (Function> & Serializable) this::getInstantDetails; + // multiple casts will make this lambda serializable - + // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 + this.details = + (Function> & Serializable) this::getInstantDetails; this.metaClient = metaClient; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java index 3a0240239..c855203a3 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java @@ -34,7 +34,7 @@ import org.apache.log4j.Logger; */ public class HoodieDefaultTimeline implements HoodieTimeline { - private final transient static Logger log = LogManager.getLogger(HoodieDefaultTimeline.class); + private static final transient Logger log = LogManager.getLogger(HoodieDefaultTimeline.class); protected Function> details; protected List instants; @@ -65,9 +65,9 @@ public class HoodieDefaultTimeline implements HoodieTimeline { @Override public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) { return new HoodieDefaultTimeline(instants.stream().filter( - s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), startTs, GREATER) && - HoodieTimeline.compareTimestamps( - s.getTimestamp(), endTs, LESSER_OR_EQUAL)), details); + s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), startTs, GREATER) + && HoodieTimeline.compareTimestamps( + s.getTimestamp(), endTs, LESSER_OR_EQUAL)), details); } @Override @@ -133,8 +133,8 @@ public class HoodieDefaultTimeline implements HoodieTimeline { @Override public boolean isBeforeTimelineStarts(String instant) { Optional firstCommit = firstInstant(); - return firstCommit.isPresent() && - HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER); + return firstCommit.isPresent() + && HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java index 1891c9807..59be65555 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java @@ -79,25 +79,20 @@ public class HoodieInstant implements Serializable { */ public String getFileName() { if (HoodieTimeline.COMMIT_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightCommitFileName(timestamp) : - HoodieTimeline.makeCommitFileName(timestamp); + return isInflight ? HoodieTimeline.makeInflightCommitFileName(timestamp) + : HoodieTimeline.makeCommitFileName(timestamp); } else if (HoodieTimeline.CLEAN_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightCleanerFileName(timestamp) : - HoodieTimeline.makeCleanerFileName(timestamp); + return isInflight ? HoodieTimeline.makeInflightCleanerFileName(timestamp) + : HoodieTimeline.makeCleanerFileName(timestamp); } else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightRollbackFileName(timestamp) : - HoodieTimeline.makeRollbackFileName(timestamp); + return isInflight ? HoodieTimeline.makeInflightRollbackFileName(timestamp) + : HoodieTimeline.makeRollbackFileName(timestamp); } else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightSavePointFileName(timestamp) : - HoodieTimeline.makeSavePointFileName(timestamp); + return isInflight ? HoodieTimeline.makeInflightSavePointFileName(timestamp) + : HoodieTimeline.makeSavePointFileName(timestamp); } else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) { - return isInflight ? - HoodieTimeline.makeInflightDeltaFileName(timestamp) : - HoodieTimeline.makeDeltaFileName(timestamp); + return isInflight ? HoodieTimeline.makeInflightDeltaFileName(timestamp) + : HoodieTimeline.makeDeltaFileName(timestamp); } throw new IllegalArgumentException("Cannot get file name for unknown action " + action); } @@ -111,9 +106,9 @@ public class HoodieInstant implements Serializable { return false; } HoodieInstant that = (HoodieInstant) o; - return isInflight == that.isInflight && - Objects.equals(action, that.action) && - Objects.equals(timestamp, that.timestamp); + return isInflight == that.isInflight + && Objects.equals(action, that.action) + && Objects.equals(timestamp, that.timestamp); } @Override 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 60df7f471..8978bf91a 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 @@ -55,209 +55,209 @@ public class HoodieTableFileSystemView implements TableFileSystemView, TableFileSystemView.ReadOptimizedView, TableFileSystemView.RealtimeView, Serializable { - protected HoodieTableMetaClient metaClient; - // This is the commits that will be visible for all views extending this view - protected HoodieTimeline visibleActiveTimeline; + protected HoodieTableMetaClient metaClient; + // This is the commits that will be visible for all views extending this view + protected HoodieTimeline visibleActiveTimeline; - // mapping from partition paths to file groups contained within them - protected HashMap> partitionToFileGroupsMap; - // mapping from file id to the file group. - protected HashMap fileGroupMap; + // mapping from partition paths to file groups contained within them + protected HashMap> partitionToFileGroupsMap; + // mapping from file id to the file group. + protected HashMap fileGroupMap; - /** - * Create a file system view, as of the given timeline - */ - public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, - HoodieTimeline visibleActiveTimeline) { - this.metaClient = metaClient; - this.visibleActiveTimeline = visibleActiveTimeline; - this.fileGroupMap = new HashMap<>(); - this.partitionToFileGroupsMap = new HashMap<>(); + /** + * Create a file system view, as of the given timeline + */ + public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, + HoodieTimeline visibleActiveTimeline) { + this.metaClient = metaClient; + this.visibleActiveTimeline = visibleActiveTimeline; + this.fileGroupMap = new HashMap<>(); + this.partitionToFileGroupsMap = new HashMap<>(); + } + + + /** + * Create a file system view, as of the given timeline, with the provided file statuses. + */ + public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, + HoodieTimeline visibleActiveTimeline, + FileStatus[] fileStatuses) { + this(metaClient, visibleActiveTimeline); + addFilesToView(fileStatuses); + } + + + /** + * This method is only used when this object is deserialized in a spark executor. + * + * @deprecated + */ + private void readObject(java.io.ObjectInputStream in) + throws IOException, ClassNotFoundException { + in.defaultReadObject(); + } + + private void writeObject(java.io.ObjectOutputStream out) + throws IOException { + out.defaultWriteObject(); + } + + /** + * Adds the provided statuses into the file system view, and also caches it inside this object. + */ + private List addFilesToView(FileStatus[] statuses) { + Map, List> dataFiles = convertFileStatusesToDataFiles( + statuses) + .collect(Collectors.groupingBy((dataFile) -> { + String partitionPathStr = FSUtils.getRelativePartitionPath( + new Path(metaClient.getBasePath()), + dataFile.getFileStatus().getPath().getParent()); + return Pair.of(partitionPathStr, dataFile.getFileId()); + })); + Map, List> logFiles = convertFileStatusesToLogFiles( + statuses) + .collect(Collectors.groupingBy((logFile) -> { + String partitionPathStr = FSUtils.getRelativePartitionPath( + new Path(metaClient.getBasePath()), + logFile.getPath().getParent()); + return Pair.of(partitionPathStr, logFile.getFileId()); + })); + + Set> fileIdSet = new HashSet<>(dataFiles.keySet()); + fileIdSet.addAll(logFiles.keySet()); + + List fileGroups = new ArrayList<>(); + fileIdSet.forEach(pair -> { + HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(), + visibleActiveTimeline); + if (dataFiles.containsKey(pair)) { + dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile)); + } + if (logFiles.containsKey(pair)) { + logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile)); + } + fileGroups.add(group); + }); + + // add to the cache. + fileGroups.forEach(group -> { + fileGroupMap.put(group.getId(), group); + if (!partitionToFileGroupsMap.containsKey(group.getPartitionPath())) { + partitionToFileGroupsMap.put(group.getPartitionPath(), new ArrayList<>()); + } + partitionToFileGroupsMap.get(group.getPartitionPath()).add(group); + }); + + return fileGroups; + } + + private Stream convertFileStatusesToDataFiles(FileStatus[] statuses) { + Predicate roFilePredicate = fileStatus -> + fileStatus.getPath().getName() + .contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()); + return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new); + } + + private Stream convertFileStatusesToLogFiles(FileStatus[] statuses) { + Predicate rtFilePredicate = fileStatus -> + fileStatus.getPath().getName() + .contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension()); + return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new); + } + + @Override + public Stream getLatestDataFiles(final String partitionPath) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getLatestDataFile()) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } + + @Override + public Stream getLatestDataFiles() { + return fileGroupMap.values().stream() + .map(fileGroup -> fileGroup.getLatestDataFile()) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } + + @Override + public Stream getLatestDataFilesBeforeOrOn(String partitionPath, + String maxCommitTime) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getLatestDataFileBeforeOrOn(maxCommitTime)) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } + + @Override + public Stream getLatestDataFilesInRange(List commitsToReturn) { + return fileGroupMap.values().stream() + .map(fileGroup -> fileGroup.getLatestDataFileInRange(commitsToReturn)) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } + + @Override + public Stream getAllDataFiles(String partitionPath) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getAllDataFiles()) + .flatMap(dataFileList -> dataFileList); + } + + @Override + public Stream getLatestFileSlices(String partitionPath) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getLatestFileSlice()) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } + + @Override + public Stream getLatestFileSlicesBeforeOrOn(String partitionPath, + String maxCommitTime) { + return getAllFileGroups(partitionPath) + .map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime)) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } + + @Override + public Stream getLatestFileSliceInRange(List commitsToReturn) { + return fileGroupMap.values().stream() + .map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn)) + .filter(dataFileOpt -> dataFileOpt.isPresent()) + .map(Optional::get); + } + + @Override + public Stream getAllFileSlices(String partitionPath) { + return getAllFileGroups(partitionPath) + .map(group -> group.getAllFileSlices()) + .flatMap(sliceList -> sliceList); + } + + /** + * Given a partition path, obtain all filegroups within that. All methods, that work at the + * partition level go through this. + */ + @Override + public Stream getAllFileGroups(String partitionPathStr) { + // return any previously fetched groups. + if (partitionToFileGroupsMap.containsKey(partitionPathStr)) { + return partitionToFileGroupsMap.get(partitionPathStr).stream(); } - - /** - * Create a file system view, as of the given timeline, with the provided file statuses. - */ - public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, - HoodieTimeline visibleActiveTimeline, - FileStatus[] fileStatuses) { - this(metaClient, visibleActiveTimeline); - addFilesToView(fileStatuses); - } - - - /** - * This method is only used when this object is deserialized in a spark executor. - * - * @deprecated - */ - private void readObject(java.io.ObjectInputStream in) - throws IOException, ClassNotFoundException { - in.defaultReadObject(); - } - - private void writeObject(java.io.ObjectOutputStream out) - throws IOException { - out.defaultWriteObject(); - } - - /** - * Adds the provided statuses into the file system view, and also caches it inside this object. - */ - private List addFilesToView(FileStatus[] statuses) { - Map, List> dataFiles = convertFileStatusesToDataFiles( - statuses) - .collect(Collectors.groupingBy((dataFile) -> { - String partitionPathStr = FSUtils.getRelativePartitionPath( - new Path(metaClient.getBasePath()), - dataFile.getFileStatus().getPath().getParent()); - return Pair.of(partitionPathStr, dataFile.getFileId()); - })); - Map, List> logFiles = convertFileStatusesToLogFiles( - statuses) - .collect(Collectors.groupingBy((logFile) -> { - String partitionPathStr = FSUtils.getRelativePartitionPath( - new Path(metaClient.getBasePath()), - logFile.getPath().getParent()); - return Pair.of(partitionPathStr, logFile.getFileId()); - })); - - Set> fileIdSet = new HashSet<>(dataFiles.keySet()); - fileIdSet.addAll(logFiles.keySet()); - - List fileGroups = new ArrayList<>(); - fileIdSet.forEach(pair -> { - HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(), - visibleActiveTimeline); - if (dataFiles.containsKey(pair)) { - dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile)); - } - if (logFiles.containsKey(pair)) { - logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile)); - } - fileGroups.add(group); - }); - - // add to the cache. - fileGroups.forEach(group -> { - fileGroupMap.put(group.getId(), group); - if (!partitionToFileGroupsMap.containsKey(group.getPartitionPath())) { - partitionToFileGroupsMap.put(group.getPartitionPath(), new ArrayList<>()); - } - partitionToFileGroupsMap.get(group.getPartitionPath()).add(group); - }); - - return fileGroups; - } - - private Stream convertFileStatusesToDataFiles(FileStatus[] statuses) { - Predicate roFilePredicate = fileStatus -> - fileStatus.getPath().getName() - .contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()); - return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new); - } - - private Stream convertFileStatusesToLogFiles(FileStatus[] statuses) { - Predicate rtFilePredicate = fileStatus -> - fileStatus.getPath().getName() - .contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension()); - return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new); - } - - @Override - public Stream getLatestDataFiles(final String partitionPath) { - return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestDataFile()) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } - - @Override - public Stream getLatestDataFiles() { - return fileGroupMap.values().stream() - .map(fileGroup -> fileGroup.getLatestDataFile()) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } - - @Override - public Stream getLatestDataFilesBeforeOrOn(String partitionPath, - String maxCommitTime) { - return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestDataFileBeforeOrOn(maxCommitTime)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } - - @Override - public Stream getLatestDataFilesInRange(List commitsToReturn) { - return fileGroupMap.values().stream() - .map(fileGroup -> fileGroup.getLatestDataFileInRange(commitsToReturn)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } - - @Override - public Stream getAllDataFiles(String partitionPath) { - return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getAllDataFiles()) - .flatMap(dataFileList -> dataFileList); - } - - @Override - public Stream getLatestFileSlices(String partitionPath) { - return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestFileSlice()) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } - - @Override - public Stream getLatestFileSlicesBeforeOrOn(String partitionPath, - String maxCommitTime) { - return getAllFileGroups(partitionPath) - .map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } - - @Override - public Stream getLatestFileSliceInRange(List commitsToReturn) { - return fileGroupMap.values().stream() - .map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn)) - .filter(dataFileOpt -> dataFileOpt.isPresent()) - .map(Optional::get); - } - - @Override - public Stream getAllFileSlices(String partitionPath) { - return getAllFileGroups(partitionPath) - .map(group -> group.getAllFileSlices()) - .flatMap(sliceList -> sliceList); - } - - /** - * Given a partition path, obtain all filegroups within that. All methods, that work at the - * partition level go through this. - */ - @Override - public Stream getAllFileGroups(String partitionPathStr) { - // return any previously fetched groups. - if (partitionToFileGroupsMap.containsKey(partitionPathStr)) { - return partitionToFileGroupsMap.get(partitionPathStr).stream(); - } - - try { - // Create the path if it does not exist already - Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr); - FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath); - FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath); - List fileGroups = addFilesToView(statuses); - return fileGroups.stream(); - } catch (IOException e) { - throw new HoodieIOException( - "Failed to list data files in partition " + partitionPathStr, e); - } + try { + // Create the path if it does not exist already + Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr); + FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath); + FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath); + List fileGroups = addFilesToView(statuses); + return fileGroups.stream(); + } catch (IOException e) { + throw new HoodieIOException( + "Failed to list data files in partition " + partitionPathStr, e); } + } } 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 31fac7355..e837285b1 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 @@ -94,8 +94,10 @@ public class FSUtils { return String.format("%s_%d_%s.parquet", fileId, taskPartitionId, commitTime); } - public static String makeTempDataFileName(String partitionPath, String commitTime, int taskPartitionId, String fileId, int stageId, long taskAttemptId) { - return String.format("%s_%s_%d_%s_%d_%d.parquet", partitionPath.replace("/", "-"), fileId, taskPartitionId, commitTime, stageId, taskAttemptId); + public static String makeTempDataFileName(String partitionPath, String commitTime, + int taskPartitionId, String fileId, int stageId, long taskAttemptId) { + return String.format("%s_%s_%d_%s_%d_%d.parquet", partitionPath.replace("/", "-"), fileId, + taskPartitionId, commitTime, stageId, taskAttemptId); } public static String maskWithoutFileId(String commitTime, int taskPartitionId) { @@ -299,7 +301,8 @@ public class FSUtils { Optional currentVersion = getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime); // handle potential overflow - return (currentVersion.isPresent()) ? currentVersion.get() + 1 : HoodieLogFile.LOGFILE_BASE_VERSION; + return (currentVersion.isPresent()) ? currentVersion.get() + 1 + : HoodieLogFile.LOGFILE_BASE_VERSION; } public static int getDefaultBufferSize(final FileSystem fs) { @@ -331,7 +334,8 @@ public class FSUtils { if (recovered) { break; } - // Sleep for 1 second before trying again. Typically it takes about 2-3 seconds to recover under default settings + // Sleep for 1 second before trying again. Typically it takes about 2-3 seconds to recover + // under default settings Thread.sleep(1000); } return recovered; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java index 519ce7b60..09d1a2cab 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java @@ -45,11 +45,11 @@ import org.apache.avro.io.EncoderFactory; public class HoodieAvroUtils { // All metadata fields are optional strings. - private final static Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList( + private static final Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList( Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING))); - private final static Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); + private static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); /** * Convert a given avro record to bytes diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java index ecbaad1c9..f4215c804 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java @@ -50,7 +50,7 @@ public class ParquetUtils { /** * Read the rowKey list from the given parquet file. * - * @param filePath The parquet file path. + * @param filePath The parquet file path. * @param configuration configuration to build fs object */ public static Set readRowKeysFromParquet(Configuration configuration, Path filePath) { @@ -116,8 +116,8 @@ public class ParquetUtils { if (metadata.containsKey(footerName)) { footerVals.add(metadata.get(footerName)); } else { - throw new MetadataNotFoundException("Could not find index in Parquet footer. " + - "Looked for key " + footerName + " in " + parquetFilePath); + throw new MetadataNotFoundException("Could not find index in Parquet footer. " + + "Looked for key " + footerName + " in " + parquetFilePath); } } return footerVals; @@ -146,7 +146,7 @@ public class ParquetUtils { "Could not read min/max record key out of footer correctly from %s. read) : %s", parquetFilePath, minMaxKeys)); } - return new String[]{minMaxKeys.get(0), minMaxKeys.get(1)}; + return new String[] {minMaxKeys.get(0), minMaxKeys.get(1)}; } /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java index 629935ee9..1d5d3b58f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java @@ -55,7 +55,8 @@ public class ReflectionUtils { } return (T) clazzCache.get(recordPayloadClass).getConstructor(constructorArgTypes) .newInstance(payloadArgs); - } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + } catch (InstantiationException | IllegalAccessException + | InvocationTargetException | NoSuchMethodException e) { throw new HoodieException("Unable to instantiate payload class ", e); } catch (ClassNotFoundException e) { throw new HoodieException("Unable to instantiate payload class ", e); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java index bdd5e6921..94aa1758d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.common.util; import com.uber.hoodie.common.model.HoodieKey; @@ -22,39 +23,28 @@ import com.uber.hoodie.common.util.collection.DiskBasedMap; import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream; import com.uber.hoodie.exception.HoodieCorruptedDataException; -import org.apache.avro.generic.GenericRecord; - import java.io.IOException; import java.io.RandomAccessFile; import java.util.Optional; import java.util.zip.CRC32; +import org.apache.avro.generic.GenericRecord; public class SpillableMapUtils { /** * Using the schema and payload class, read and convert the bytes on disk to a HoodieRecord - * - * @param file - * @param valuePosition - * @param valueLength - * @return - * @throws IOException */ - public static byte[] readBytesFromDisk(RandomAccessFile file, long valuePosition, int valueLength) throws IOException { + public static byte[] readBytesFromDisk(RandomAccessFile file, long valuePosition, int valueLength) + throws IOException { DiskBasedMap.FileEntry fileEntry = readInternal(file, valuePosition, valueLength); return fileEntry.getValue(); } /** * |crc|timestamp|sizeOfKey|SizeOfValue|key|value| - * - * @param file - * @param valuePosition - * @param valueLength - * @return - * @throws IOException */ - private static DiskBasedMap.FileEntry readInternal(RandomAccessFile file, long valuePosition, int valueLength) throws IOException { + private static DiskBasedMap.FileEntry readInternal(RandomAccessFile file, long valuePosition, + int valueLength) throws IOException { file.seek(valuePosition); long crc = file.readLong(); long timestamp = file.readLong(); @@ -69,27 +59,23 @@ public class SpillableMapUtils { file.read(value, 0, valueSize); long crcOfReadValue = generateChecksum(value); if (!(crc == crcOfReadValue)) { - throw new HoodieCorruptedDataException("checksum of payload written to external disk does not match, " + - "data may be corrupted"); + throw new HoodieCorruptedDataException("checksum of payload written to external disk does not match, " + + "data may be corrupted"); } return new DiskBasedMap.FileEntry(crc, keySize, valueSize, key, value, timestamp); } /** - * Write Value and other metadata necessary to disk. Each entry has the following sequence of data - *

+ * Write Value and other metadata necessary to disk. Each entry has the following sequence of data

* |crc|timestamp|sizeOfKey|SizeOfValue|key|value| - * - * @param outputStream - * @param fileEntry - * @return - * @throws IOException */ - public static long spillToDisk(SizeAwareDataOutputStream outputStream, DiskBasedMap.FileEntry fileEntry) throws IOException { + public static long spillToDisk(SizeAwareDataOutputStream outputStream, + DiskBasedMap.FileEntry fileEntry) throws IOException { return spill(outputStream, fileEntry); } - private static long spill(SizeAwareDataOutputStream outputStream, DiskBasedMap.FileEntry fileEntry) + private static long spill(SizeAwareDataOutputStream outputStream, + DiskBasedMap.FileEntry fileEntry) throws IOException { outputStream.writeLong(fileEntry.getCrc()); outputStream.writeLong(fileEntry.getTimestamp()); @@ -102,9 +88,6 @@ public class SpillableMapUtils { /** * Generate a checksum for a given set of bytes - * - * @param data - * @return */ public static long generateChecksum(byte[] data) { CRC32 crc = new CRC32(); @@ -113,13 +96,8 @@ public class SpillableMapUtils { } /** - * Compute a bytes representation of the payload by serializing the contents - * This is used to estimate the size of the payload (either in memory or when written to disk) - * - * @param - * @param value - * @return - * @throws IOException + * Compute a bytes representation of the payload by serializing the contents This is used to estimate the size of the + * payload (either in memory or when written to disk) */ public static long computePayloadSize(R value, Converter valueConverter) throws IOException { return valueConverter.sizeEstimate(value); @@ -127,12 +105,6 @@ public class SpillableMapUtils { /** * Utility method to convert bytes to HoodieRecord using schema and payload class - * - * @param rec - * @param payloadClazz - * @param - * @return - * @throws IOException */ public static R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz) { String recKey = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java index 27ee0fba2..0c3ab8626 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java @@ -22,9 +22,6 @@ import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStre import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieNotSupportedException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -40,18 +37,25 @@ import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** - * This class provides a disk spillable only map implementation. All of the data is - * currenly written to one file, without any rollover support. It uses the following : - * 1) An in-memory map that tracks the key-> latest ValueMetadata. - * 2) Current position in the file - * NOTE : Only String.class type supported for Key + * This class provides a disk spillable only map implementation. All of the data is currenly written to one file, + * without any rollover support. It uses the following : 1) An in-memory map that tracks the key-> latest ValueMetadata. + * 2) Current position in the file NOTE : Only String.class type supported for Key */ -final public class DiskBasedMap implements Map { +public final class DiskBasedMap implements Map { + private static final Logger log = LogManager.getLogger(DiskBasedMap.class); + // Default file path prefix to put the spillable file + private static String DEFAULT_BASE_FILE_PATH = "/tmp/"; // Stores the key and corresponding value's latest metadata spilled to disk - final private Map valueMetadataMap; + private final Map valueMetadataMap; + // Key converter to convert key type to bytes + private final Converter keyConverter; + // Value converter to convert value type to bytes + private final Converter valueConverter; // Read only file access to be able to seek to random positions to readFromDisk values private RandomAccessFile readOnlyFileHandle; // Write only OutputStream to be able to ONLY append to the file @@ -63,100 +67,7 @@ final public class DiskBasedMap implements Map { private AtomicLong filePosition; // FilePath to store the spilled data private String filePath; - // Default file path prefix to put the spillable file - private static String DEFAULT_BASE_FILE_PATH = "/tmp/"; - // Key converter to convert key type to bytes - final private Converter keyConverter; - // Value converter to convert value type to bytes - final private Converter valueConverter; - private static Logger log = LogManager.getLogger(DiskBasedMap.class); - - - public final class ValueMetadata { - - // FilePath to store the spilled data - private String filePath; - // Size (numberOfBytes) of the value written to disk - private Integer sizeOfValue; - // FilePosition of the value written to disk - private Long offsetOfValue; - // Current timestamp when the value was written to disk - private Long timestamp; - - protected ValueMetadata(String filePath, int sizeOfValue, long offsetOfValue, long timestamp) { - this.filePath = filePath; - this.sizeOfValue = sizeOfValue; - this.offsetOfValue = offsetOfValue; - this.timestamp = timestamp; - } - - public String getFilePath() { - return filePath; - } - - public int getSizeOfValue() { - return sizeOfValue; - } - - public Long getOffsetOfValue() { - return offsetOfValue; - } - - public long getTimestamp() { - return timestamp; - } - } - - public static final class FileEntry { - - // Checksum of the value written to disk, compared during every readFromDisk to make sure no corruption - private Long crc; - // Size (numberOfBytes) of the key written to disk - private Integer sizeOfKey; - // Size (numberOfBytes) of the value written to disk - private Integer sizeOfValue; - // Actual key - private byte[] key; - // Actual value - private byte[] value; - // Current timestamp when the value was written to disk - private Long timestamp; - - public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte[] key, byte[] value, - long timestamp) { - this.crc = crc; - this.sizeOfKey = sizeOfKey; - this.sizeOfValue = sizeOfValue; - this.key = key; - this.value = value; - this.timestamp = timestamp; - } - - public long getCrc() { - return crc; - } - - public int getSizeOfKey() { - return sizeOfKey; - } - - public int getSizeOfValue() { - return sizeOfValue; - } - - public byte[] getKey() { - return key; - } - - public byte[] getValue() { - return value; - } - - public long getTimestamp() { - return timestamp; - } - } protected DiskBasedMap(Optional baseFilePath, Converter keyConverter, Converter valueConverter) throws IOException { @@ -193,8 +104,8 @@ final public class DiskBasedMap implements Map { } /** - * Register shutdown hook to force flush contents of the data written to FileOutputStream - * from OS page cache (typically 4 KB) to disk + * Register shutdown hook to force flush contents of the data written to FileOutputStream from OS page cache + * (typically 4 KB) to disk */ private void addShutDownHook() { Runtime.getRuntime().addShutdownHook(new Thread() { @@ -324,4 +235,89 @@ final public class DiskBasedMap implements Map { } return entrySet; } + + public static final class FileEntry { + + // Checksum of the value written to disk, compared during every readFromDisk to make sure no corruption + private Long crc; + // Size (numberOfBytes) of the key written to disk + private Integer sizeOfKey; + // Size (numberOfBytes) of the value written to disk + private Integer sizeOfValue; + // Actual key + private byte[] key; + // Actual value + private byte[] value; + // Current timestamp when the value was written to disk + private Long timestamp; + + public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte[] key, byte[] value, + long timestamp) { + this.crc = crc; + this.sizeOfKey = sizeOfKey; + this.sizeOfValue = sizeOfValue; + this.key = key; + this.value = value; + this.timestamp = timestamp; + } + + public long getCrc() { + return crc; + } + + public int getSizeOfKey() { + return sizeOfKey; + } + + public int getSizeOfValue() { + return sizeOfValue; + } + + public byte[] getKey() { + return key; + } + + public byte[] getValue() { + return value; + } + + public long getTimestamp() { + return timestamp; + } + } + + public final class ValueMetadata { + + // FilePath to store the spilled data + private String filePath; + // Size (numberOfBytes) of the value written to disk + private Integer sizeOfValue; + // FilePosition of the value written to disk + private Long offsetOfValue; + // Current timestamp when the value was written to disk + private Long timestamp; + + protected ValueMetadata(String filePath, int sizeOfValue, long offsetOfValue, long timestamp) { + this.filePath = filePath; + this.sizeOfValue = sizeOfValue; + this.offsetOfValue = offsetOfValue; + this.timestamp = timestamp; + } + + public String getFilePath() { + return filePath; + } + + public int getSizeOfValue() { + return sizeOfValue; + } + + public Long getOffsetOfValue() { + return offsetOfValue; + } + + public long getTimestamp() { + return timestamp; + } + } } \ No newline at end of file diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java index 3d390690e..b3ff515c0 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java @@ -18,11 +18,7 @@ package com.uber.hoodie.common.util.collection; import com.twitter.common.objectsize.ObjectSizeCalculator; import com.uber.hoodie.common.util.collection.converter.Converter; -import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieNotSupportedException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - import java.io.IOException; import java.util.Collection; import java.util.HashMap; @@ -31,43 +27,43 @@ import java.util.Iterator; import java.util.Map; import java.util.Optional; import java.util.Set; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; /** - * An external map that spills content to disk when there is insufficient space for it to grow.

- * This map holds 2 types of data structures :

(1) Key-Value pairs in a in-memory map (2) - * Key-ValueMetadata pairs in an in-memory map which keeps a marker to the values spilled to disk - *

NOTE : Values are only appended to disk. If a remove() is called, the entry is marked removed - * from the in-memory key-valueMetadata map but it's values will be lying around in the temp file on - * disk until the file is cleaned.

The setting of the spill threshold faces the following - * trade-off: If the spill threshold is too high, the in-memory map may occupy more memory than is - * available, resulting in OOM. However, if the spill threshold is too low, we spill frequently and - * incur unnecessary disk writes. + * An external map that spills content to disk when there is insufficient space for it to grow.

This map holds 2 + * types of data structures :

(1) Key-Value pairs in a in-memory map (2) Key-ValueMetadata pairs in an in-memory map + * which keeps a marker to the values spilled to disk

NOTE : Values are only appended to disk. If a remove() is + * called, the entry is marked removed from the in-memory key-valueMetadata map but it's values will be lying around in + * the temp file on disk until the file is cleaned.

The setting of the spill threshold faces the following + * trade-off: If the spill threshold is too high, the in-memory map may occupy more memory than is available, resulting + * in OOM. However, if the spill threshold is too low, we spill frequently and incur unnecessary disk writes. */ public class ExternalSpillableMap implements Map { // Find the actual estimated payload size after inserting N records - final private static int NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE = 100; + private static final int NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE = 100; + private static final Logger log = LogManager.getLogger(ExternalSpillableMap.class); // maximum space allowed in-memory for this map - final private long maxInMemorySizeInBytes; + private final long maxInMemorySizeInBytes; + // Map to store key-values in memory until it hits maxInMemorySizeInBytes + private final Map inMemoryMap; + // Map to store key-valuemetadata important to find the values spilled to disk + private final DiskBasedMap diskBasedMap; + // TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and + // incorrect payload estimation + private final Double sizingFactorForInMemoryMap = 0.8; + // Key converter to convert key type to bytes + private final Converter keyConverter; + // Value converter to convert value type to bytes + private final Converter valueConverter; // current space occupied by this map in-memory private Long currentInMemoryMapSize; - // Map to store key-values in memory until it hits maxInMemorySizeInBytes - final private Map inMemoryMap; - // Map to store key-valuemetadata important to find the values spilled to disk - final private DiskBasedMap diskBasedMap; // An estimate of the size of each payload written to this map private volatile long estimatedPayloadSize = 0; - // TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and incorrect payload estimation - final private Double sizingFactorForInMemoryMap = 0.8; - // Key converter to convert key type to bytes - final private Converter keyConverter; - // Value converter to convert value type to bytes - final private Converter valueConverter; // Flag to determine whether to stop re-estimating payload size private boolean shouldEstimatePayloadSize = true; - private static Logger log = LogManager.getLogger(ExternalSpillableMap.class); - public ExternalSpillableMap(Long maxInMemorySizeInBytes, Optional baseFilePath, Converter keyConverter, Converter valueConverter) throws IOException { this.inMemoryMap = new HashMap<>(); @@ -153,15 +149,14 @@ public class ExternalSpillableMap implements Map { this.estimatedPayloadSize = keyConverter.sizeEstimate(key) + valueConverter.sizeEstimate(value); log.info("Estimated Payload size => " + estimatedPayloadSize); - } - else if(shouldEstimatePayloadSize && - inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { + } else if (shouldEstimatePayloadSize + && inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { // Re-estimate the size of a record by calculating the size of the entire map containing // N entries and then dividing by the number of entries present (N). This helps to get a // correct estimation of the size of each record in the JVM. long totalMapSize = ObjectSizeCalculator.getObjectSize(inMemoryMap); this.currentInMemoryMapSize = totalMapSize; - this.estimatedPayloadSize = totalMapSize/inMemoryMap.size(); + this.estimatedPayloadSize = totalMapSize / inMemoryMap.size(); shouldEstimatePayloadSize = false; log.info("New Estimated Payload size => " + this.estimatedPayloadSize); } @@ -227,9 +222,8 @@ public class ExternalSpillableMap implements Map { } /** - * Iterator that wraps iterating over all the values for this map - * 1) inMemoryIterator - Iterates over all the data in-memory map - * 2) diskLazyFileIterator - Iterates over all the data spilled to disk + * Iterator that wraps iterating over all the values for this map 1) inMemoryIterator - Iterates over all the data + * in-memory map 2) diskLazyFileIterator - Iterates over all the data spilled to disk */ private class IteratorWrapper implements Iterator { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java index 78eaa3f34..0d53e2659 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java @@ -20,7 +20,6 @@ import com.uber.hoodie.common.util.SpillableMapUtils; import com.uber.hoodie.common.util.collection.converter.Converter; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; - import java.io.IOException; import java.io.RandomAccessFile; import java.util.Iterator; @@ -29,8 +28,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; /** - * Iterable to lazily fetch values spilled to disk. - * This class uses RandomAccessFile to randomly access the position of + * Iterable to lazily fetch values spilled to disk. This class uses RandomAccessFile to randomly access the position of * the latest value for a key spilled to disk and returns the result. */ public class LazyFileIterable implements Iterable { @@ -63,9 +61,9 @@ public class LazyFileIterable implements Iterable { */ public class LazyFileIterator implements Iterator { + private final Converter valueConverter; private RandomAccessFile readOnlyFileHandle; private Iterator> metadataIterator; - private final Converter valueConverter; public LazyFileIterator(RandomAccessFile file, Map map, Converter valueConverter) throws IOException { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java index 73dc0541a..88bf7fcd0 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/converter/HoodieRecordConverter.java @@ -22,14 +22,9 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.ReflectionUtils; -import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieNotSerializableException; import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.Field; -import java.util.Arrays; import java.util.Optional; -import java.util.stream.Stream; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.commons.lang3.SerializationUtils; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java index 8280dda3f..df31305de 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/io/storage/SizeAwareDataOutputStream.java @@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicLong; * Wrapper for DataOutpuStream to keep track of number of bytes written */ public class SizeAwareDataOutputStream { + // Actual outpuStream private DataOutputStream outputStream; // Counter to keep track of number of bytes written @@ -45,12 +46,12 @@ public class SizeAwareDataOutputStream { outputStream.writeInt(v); } - public void write(byte [] v) throws IOException { + public void write(byte[] v) throws IOException { size.addAndGet(v.length); outputStream.write(v); } - public void write(byte [] v, int offset, int len) throws IOException { + public void write(byte[] v, int offset, int len) throws IOException { size.addAndGet(len + offset); outputStream.write(v, offset, len); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java index 3c10a1537..773dbfc92 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieCorruptedDataException.java @@ -17,7 +17,8 @@ package com.uber.hoodie.exception; /** - *

Exception thrown when any data corruption happens when reading/writing from temporary disk

+ *

Exception thrown when any data corruption happens when reading/writing from temporary disk + *

*/ public class HoodieCorruptedDataException extends HoodieException { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java b/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java index 6c024a897..dbc3567ff 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java @@ -19,9 +19,7 @@ package com.uber.hoodie.common; import java.io.IOException; import org.junit.Test; -public class - -TestBloomFilter { +public class TestBloomFilter { @Test public void testAddKey() { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java index a90739f75..c5d6e1e93 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java @@ -16,7 +16,6 @@ package com.uber.hoodie.common.minicluster; - import com.google.common.base.Preconditions; import com.google.common.io.Files; import com.uber.hoodie.common.model.HoodieTestUtils; @@ -63,25 +62,23 @@ public class HdfsTestService { } public MiniDFSCluster start(boolean format) throws IOException { - Preconditions - .checkState(workDir != null, "The work dir must be set before starting cluster."); + Preconditions.checkState(workDir != null, "The work dir must be set before starting cluster."); hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); // If clean, then remove the work dir so we can start fresh. String localDFSLocation = getDFSLocation(workDir); if (format) { - logger.info( - "Cleaning HDFS cluster data at: " + localDFSLocation + " and starting fresh."); + logger.info("Cleaning HDFS cluster data at: " + localDFSLocation + " and starting fresh."); File file = new File(localDFSLocation); FileUtils.deleteDirectory(file); } // Configure and start the HDFS cluster // boolean format = shouldFormatDFSCluster(localDFSLocation, clean); - hadoopConf = configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort, - namenodeHttpPort, datanodePort, datanodeIpcPort, datanodeHttpPort); - miniDfsCluster = new MiniDFSCluster.Builder(hadoopConf).numDataNodes(1).format(format) - .checkDataNodeAddrConfig(true).checkDataNodeHostConfig(true).build(); + hadoopConf = configureDFSCluster(hadoopConf, localDFSLocation, bindIP, namenodeRpcPort, namenodeHttpPort, + datanodePort, datanodeIpcPort, datanodeHttpPort); + miniDfsCluster = new MiniDFSCluster.Builder(hadoopConf).numDataNodes(1).format(format).checkDataNodeAddrConfig(true) + .checkDataNodeHostConfig(true).build(); logger.info("HDFS Minicluster service started."); return miniDfsCluster; } @@ -96,8 +93,7 @@ public class HdfsTestService { /** * Get the location on the local FS where we store the HDFS data. * - * @param baseFsLocation The base location on the local filesystem we have write access to create - * dirs. + * @param baseFsLocation The base location on the local filesystem we have write access to create dirs. * @return The location for HDFS data. */ private static String getDFSLocation(String baseFsLocation) { @@ -105,11 +101,11 @@ public class HdfsTestService { } /** - * Returns true if we should format the DFS Cluster. We'll format if clean is true, or if the - * dfsFsLocation does not exist. + * Returns true if we should format the DFS Cluster. We'll format if clean is true, or if the dfsFsLocation does not + * exist. * * @param localDFSLocation The location on the local FS to hold the HDFS metadata and block data - * @param clean Specifies if we want to start a clean cluster + * @param clean Specifies if we want to start a clean cluster * @return Returns true if we should format a DFSCluster, otherwise false */ private static boolean shouldFormatDFSCluster(String localDFSLocation, boolean clean) { @@ -124,14 +120,13 @@ public class HdfsTestService { /** * Configure the DFS Cluster before launching it. * - * @param config The already created Hadoop configuration we'll further configure for HDFS + * @param config The already created Hadoop configuration we'll further configure for HDFS * @param localDFSLocation The location on the local filesystem where cluster data is stored - * @param bindIP An IP address we want to force the datanode and namenode to bind to. + * @param bindIP An IP address we want to force the datanode and namenode to bind to. * @return The updated Configuration object. */ - private static Configuration configureDFSCluster(Configuration config, String localDFSLocation, - String bindIP, int namenodeRpcPort, int namenodeHttpPort, int datanodePort, - int datanodeIpcPort, int datanodeHttpPort) { + private static Configuration configureDFSCluster(Configuration config, String localDFSLocation, String bindIP, + int namenodeRpcPort, int namenodeHttpPort, int datanodePort, int datanodeIpcPort, int datanodeHttpPort) { logger.info("HDFS force binding to ip: " + bindIP); config.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "hdfs://" + bindIP + ":" + namenodeRpcPort); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java index 290064e19..d47bfa329 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/ZookeeperTestService.java @@ -43,11 +43,10 @@ import org.slf4j.LoggerFactory; *

* 2. It will only attempt to bind to the port specified, and will fail if it can't. *

- * 3. The startup method now takes a bindAddress, which allows us to configure which IP the ZK - * server binds to. This was not configurable in the original class. + * 3. The startup method now takes a bindAddress, which allows us to configure which IP the ZK server binds to. This was + * not configurable in the original class. *

- * 4. The ZK cluster will re-use a data dir on the local filesystem if it already exists instead of - * blowing it away. + * 4. The ZK cluster will re-use a data dir on the local filesystem if it already exists instead of blowing it away. */ public class ZookeeperTestService { @@ -83,8 +82,7 @@ public class ZookeeperTestService { } public ZooKeeperServer start() throws IOException, InterruptedException { - Preconditions.checkState(workDir != null, - "The localBaseFsLocation must be set before starting cluster."); + Preconditions.checkState(workDir != null, "The localBaseFsLocation must be set before starting cluster."); setupTestEnv(); stop(); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java index 984fb3a41..bc40108e9 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java @@ -47,7 +47,8 @@ public class AvroBinaryTestPayload implements HoodieRecordPayload { } @Override - public Optional combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { + public Optional combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) + throws IOException { return getInsertValue(schema); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index bd698e7c8..361bd81e7 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -80,19 +80,16 @@ public class HoodieTestUtils { return new Configuration(); } - public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath) - throws IOException { + public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath) throws IOException { return initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE); } public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, - HoodieTableType tableType) - throws IOException { + HoodieTableType tableType) throws IOException { Properties properties = new Properties(); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name()); - properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, - HoodieAvroPayload.class.getName()); + properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName()); return HoodieTableMetaClient.initializePathAsHoodieDataset(hadoopConf, basePath, properties); } @@ -108,63 +105,60 @@ public class HoodieTestUtils { return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); } - public static final void createCommitFiles(String basePath, String... commitTimes) - throws IOException { + public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException { + for (String commitTime : commitTimes) { + new File( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime)) + .createNewFile(); + } + } + + public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException { + for (String commitTime : commitTimes) { + new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeInflightCommitFileName( + commitTime)).createNewFile(); + } + } + + public static final void createInflightCleanFiles(String basePath, String... commitTimes) throws IOException { for (String commitTime : commitTimes) { new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCommitFileName(commitTime)).createNewFile(); + .makeInflightCleanerFileName( + commitTime)).createNewFile(); } } - public static final void createInflightCommitFiles(String basePath, String... commitTimes) + public static final String createNewDataFile(String basePath, String partitionPath, String commitTime) throws IOException { - for (String commitTime : commitTimes) { - new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeInflightCommitFileName(commitTime)).createNewFile(); - } - } - - public static final void createInflightCleanFiles(String basePath, String... commitTimes) - throws IOException { - for (String commitTime : commitTimes) { - new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + - HoodieTimeline.makeInflightCleanerFileName(commitTime)).createNewFile(); - } - } - - public static final String createNewDataFile(String basePath, String partitionPath, - String commitTime) throws IOException { String fileID = UUID.randomUUID().toString(); return createDataFile(basePath, partitionPath, commitTime, fileID); } - public static final String createDataFile(String basePath, String partitionPath, - String commitTime, String fileID) throws IOException { + public static final String createDataFile(String basePath, String partitionPath, String commitTime, String fileID) + throws IOException { String folderPath = basePath + "/" + partitionPath + "/"; new File(folderPath).mkdirs(); - new File(folderPath + FSUtils.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID)) - .createNewFile(); + new File(folderPath + FSUtils.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID)).createNewFile(); return fileID; } - public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, - String commitTime, String fileID, Optional version) throws IOException { + public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime, + String fileID, Optional version) throws IOException { String folderPath = basePath + "/" + partitionPath + "/"; boolean makeDir = fs.mkdirs(new Path(folderPath)); if (!makeDir) { throw new IOException("cannot create directory for path " + folderPath); } - boolean createFile = fs.createNewFile(new Path(folderPath + FSUtils - .makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_TASK_PARTITIONID)))); + boolean createFile = fs.createNewFile(new Path( + folderPath + FSUtils.makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_TASK_PARTITIONID)))); if (!createFile) { - throw new IOException(StringUtils - .format("cannot create data file for commit %s and fileId %s", commitTime, fileID)); + throw new IOException( + StringUtils.format("cannot create data file for commit %s and fileId %s", commitTime, fileID)); } return fileID; } - public static final void createCompactionCommitFiles(FileSystem fs, String basePath, - String... commitTimes) + public static final void createCompactionCommitFiles(FileSystem fs, String basePath, String... commitTimes) throws IOException { for (String commitTime : commitTimes) { boolean createFile = fs.createNewFile(new Path( @@ -176,42 +170,42 @@ public class HoodieTestUtils { } } - public static final String getDataFilePath(String basePath, String partitionPath, - String commitTime, String fileID) throws IOException { + public static final String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID) + throws IOException { return basePath + "/" + partitionPath + "/" + FSUtils .makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID); } - public static final String getLogFilePath(String basePath, String partitionPath, - String commitTime, String fileID, Optional version) throws IOException { - return basePath + "/" + partitionPath + "/" + FSUtils - .makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_TASK_PARTITIONID)); + public static final String getLogFilePath(String basePath, String partitionPath, String commitTime, String fileID, + Optional version) throws IOException { + return basePath + "/" + partitionPath + "/" + FSUtils.makeLogFileName(fileID, ".log", commitTime, + version.orElse(DEFAULT_TASK_PARTITIONID)); } - public static final String getCommitFilePath(String basePath, String commitTime) + public static final String getCommitFilePath(String basePath, String commitTime) throws IOException { + return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION; + } + + public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime, String fileID) throws IOException { - return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime - + HoodieTimeline.COMMIT_EXTENSION; - } - - public static final boolean doesDataFileExist(String basePath, String partitionPath, - String commitTime, String fileID) throws IOException { return new File(getDataFilePath(basePath, partitionPath, commitTime, fileID)).exists(); } - public static final boolean doesLogFileExist(String basePath, String partitionPath, - String commitTime, String fileID, Optional version) throws IOException { + public static final boolean doesLogFileExist(String basePath, String partitionPath, String commitTime, String fileID, + Optional version) throws IOException { return new File(getLogFilePath(basePath, partitionPath, commitTime, fileID, version)).exists(); } public static final boolean doesCommitExist(String basePath, String commitTime) { - return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime - + HoodieTimeline.COMMIT_EXTENSION).exists(); + return new File( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION) + .exists(); } public static final boolean doesInflightExist(String basePath, String commitTime) { - return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime - + HoodieTimeline.INFLIGHT_EXTENSION).exists(); + return new File( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.INFLIGHT_EXTENSION) + .exists(); } public static String makeInflightTestFileName(String instant) { @@ -219,20 +213,17 @@ public class HoodieTestUtils { } public static void createCleanFiles(String basePath, String commitTime) throws IOException { - Path commitFile = - new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCleanerFileName(commitTime)); + Path commitFile = new Path( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime)); FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); FSDataOutputStream os = fs.create(commitFile, true); try { - HoodieCleanStat cleanStats = new HoodieCleanStat( - HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS, - DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)], - new ArrayList<>(), new ArrayList<>(), + HoodieCleanStat cleanStats = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS, + DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)], new ArrayList<>(), new ArrayList<>(), new ArrayList<>(), commitTime); // Create the clean metadata - HoodieCleanMetadata cleanMetadata = - AvroUtils.convertCleanMetadata(commitTime, Optional.of(0L), Arrays.asList(cleanStats)); + HoodieCleanMetadata cleanMetadata = AvroUtils.convertCleanMetadata(commitTime, Optional.of(0L), + Arrays.asList(cleanStats)); // Write empty clean metadata os.write(AvroUtils.serializeCleanMetadata(cleanMetadata).get()); } finally { @@ -249,7 +240,8 @@ public class HoodieTestUtils { } public static void assertStreamEquals(String message, Stream expected, Stream actual) { - Iterator iter1 = expected.iterator(), iter2 = actual.iterator(); + Iterator iter1 = expected.iterator(); + Iterator iter2 = actual.iterator(); while (iter1.hasNext() && iter2.hasNext()) { assertEquals(message, iter1.next(), iter2.next()); } @@ -275,8 +267,8 @@ public class HoodieTestUtils { public static void writeRecordsToLogFiles(FileSystem fs, String basePath, Schema schema, List updatedRecords) { - Map> groupedUpdated = updatedRecords.stream() - .collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation)); + Map> groupedUpdated = updatedRecords.stream().collect( + Collectors.groupingBy(HoodieRecord::getCurrentLocation)); groupedUpdated.entrySet().forEach(s -> { HoodieRecordLocation location = s.getKey(); @@ -284,12 +276,9 @@ public class HoodieTestUtils { Writer logWriter; try { - logWriter = HoodieLogFormat.newWriterBuilder() - .onParentPath(new Path(basePath, partitionPath)) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION) - .withFileId(location.getFileId()) - .overBaseCommit(location.getCommitTime()) - .withFs(fs).build(); + logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(basePath, partitionPath)) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(location.getFileId()) + .overBaseCommit(location.getCommitTime()).withFs(fs).build(); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, location.getCommitTime()); @@ -297,10 +286,7 @@ public class HoodieTestUtils { logWriter.appendBlock(new HoodieAvroDataBlock(s.getValue().stream().map(r -> { try { GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get(); - HoodieAvroUtils.addHoodieKeyToRecord(val, - r.getRecordKey(), - r.getPartitionPath(), - ""); + HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), ""); return (IndexedRecord) val; } catch (IOException e) { return null; @@ -313,8 +299,7 @@ public class HoodieTestUtils { }); } - public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath) - throws IOException { + public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath) throws IOException { RemoteIterator itr = fs.listFiles(new Path(basePath), true); List returns = Lists.newArrayList(); while (itr.hasNext()) { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java index 77c7d21de..a462a6da3 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieWriteStat.java @@ -43,10 +43,10 @@ public class TestHoodieWriteStat { Path partitionPath = new Path(basePath, partitionPathString); Path tempPath = new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME); - Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(commitTime, - taskPartitionId, fileName)); - Path tempFilePath = new Path(tempPath, FSUtils.makeTempDataFileName(partitionPathString, - commitTime, taskPartitionId, fileName, stageId, taskAttemptId)); + Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName)); + Path tempFilePath = new Path(tempPath, FSUtils + .makeTempDataFileName(partitionPathString, commitTime, taskPartitionId, + fileName, stageId, taskAttemptId)); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setPaths(basePath, finalizeFilePath, tempFilePath); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java index bd710d1a9..414166b99 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java @@ -53,25 +53,22 @@ public class HoodieTableMetaClientTest { assertEquals("Table name should be raw_trips", HoodieTestUtils.RAW_TRIPS_TEST_NAME, metaClient.getTableConfig().getTableName()); assertEquals("Basepath should be the one assigned", basePath, metaClient.getBasePath()); - assertEquals("Metapath should be ${basepath}/.hoodie", basePath + "/.hoodie", - metaClient.getMetaPath()); + assertEquals("Metapath should be ${basepath}/.hoodie", basePath + "/.hoodie", metaClient.getMetaPath()); } @Test public void checkSerDe() throws IOException, ClassNotFoundException { // check if this object is serialized and de-serialized, we are able to read from the file system - HoodieTableMetaClient deseralizedMetaClient = - HoodieTestUtils.serializeDeserialize(metaClient, HoodieTableMetaClient.class); + HoodieTableMetaClient deseralizedMetaClient = HoodieTestUtils + .serializeDeserialize(metaClient, HoodieTableMetaClient.class); assertNotNull(deseralizedMetaClient); HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline(); - HoodieInstant instant = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); commitTimeline.createInflight(instant); commitTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes())); commitTimeline = commitTimeline.reload(); HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant); - assertEquals("Commit should be 1 and completed", completedInstant, - commitTimeline.getInstants().findFirst().get()); + assertEquals("Commit should be 1 and completed", completedInstant, commitTimeline.getInstants().findFirst().get()); assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(), commitTimeline.getInstantDetails(completedInstant).get()); } @@ -82,8 +79,7 @@ public class HoodieTableMetaClientTest { HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline(); assertTrue("Should be empty commit timeline", activeCommitTimeline.empty()); - HoodieInstant instant = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); activeTimeline.createInflight(instant); activeTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes())); @@ -96,8 +92,7 @@ public class HoodieTableMetaClientTest { activeTimeline = activeTimeline.reload(); activeCommitTimeline = activeTimeline.getCommitTimeline(); assertFalse("Should be the 1 commit we made", activeCommitTimeline.empty()); - assertEquals("Commit should be 1", completedInstant, - activeCommitTimeline.getInstants().findFirst().get()); + assertEquals("Commit should be 1", completedInstant, activeCommitTimeline.getInstants().findFirst().get()); assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(), activeCommitTimeline.getInstantDetails(completedInstant).get()); } @@ -118,22 +113,16 @@ public class HoodieTableMetaClientTest { HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); - HoodieInstant instant1 = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant instant2 = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2"); - HoodieInstant instant3 = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant1 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2"); + HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); assertEquals(Lists.newArrayList(instant1, instant2, instant3), archivedTimeline.getInstants().collect(Collectors.toList())); - assertArrayEquals(new Text("data1").getBytes(), - archivedTimeline.getInstantDetails(instant1).get()); - assertArrayEquals(new Text("data2").getBytes(), - archivedTimeline.getInstantDetails(instant2).get()); - assertArrayEquals(new Text("data3").getBytes(), - archivedTimeline.getInstantDetails(instant3).get()); + assertArrayEquals(new Text("data1").getBytes(), archivedTimeline.getInstantDetails(instant1).get()); + assertArrayEquals(new Text("data2").getBytes(), archivedTimeline.getInstantDetails(instant2).get()); + assertArrayEquals(new Text("data3").getBytes(), archivedTimeline.getInstantDetails(instant3).get()); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 209a53cb4..3d37e6816 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -16,6 +16,12 @@ package com.uber.hoodie.common.table.log; +import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import com.google.common.collect.Maps; import com.uber.hoodie.common.minicluster.MiniClusterUtil; import com.uber.hoodie.common.model.HoodieArchivedLogFile; @@ -35,6 +41,17 @@ import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.exception.CorruptedLogFileException; +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -51,24 +68,6 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.IOException; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - @SuppressWarnings("Duplicates") @RunWith(Parameterized.class) public class HoodieLogFormatTest { @@ -85,9 +84,7 @@ public class HoodieLogFormatTest { @Parameterized.Parameters(name = "LogBlockReadMode") public static Collection data() { - return Arrays.asList(new Boolean[][]{ - {true},{false} - }); + return Arrays.asList(new Boolean[][] {{true}, {false}}); } @BeforeClass @@ -123,10 +120,8 @@ public class HoodieLogFormatTest { .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); assertEquals("Just created this log, size should be 0", 0, writer.getCurrentSize()); - assertTrue("Check all log files should start with a .", - writer.getLogFile().getFileName().startsWith(".")); - assertEquals("Version should be 1 for new log created", 1, - writer.getLogFile().getLogVersion()); + assertTrue("Check all log files should start with a .", writer.getLogFile().getFileName().startsWith(".")); + assertEquals("Version should be 1 for new log created", 1, writer.getLogFile().getLogVersion()); } @Test @@ -142,9 +137,8 @@ public class HoodieLogFormatTest { writer = writer.appendBlock(dataBlock); long size = writer.getCurrentSize(); assertTrue("We just wrote a block - size should be > 0", size > 0); - assertEquals( - "Write should be auto-flushed. The size reported by FileStatus and the writer should match", - size, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); + assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", size, + fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); } @@ -166,14 +160,13 @@ public class HoodieLogFormatTest { // Create a writer with the size threshold as the size we just wrote - so this has to roll writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).withSizeThreshold(size - 1).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); - assertEquals("This should be a new log file and hence size should be 0", 0, - writer.getCurrentSize()); + assertEquals("This should be a new log file and hence size should be 0", 0, writer.getCurrentSize()); assertEquals("Version should be rolled to 2", 2, writer.getLogFile().getLogVersion()); writer.close(); } @@ -193,32 +186,30 @@ public class HoodieLogFormatTest { writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 = writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1); - assertEquals( - "Write should be auto-flushed. The size reported by FileStatus and the writer should match", - size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); + assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", size2, + fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); // Close and Open again and append 100 more records writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size3 = writer.getCurrentSize(); assertTrue("We just wrote a new block - size3 should be > size2", size3 > size2); - assertEquals( - "Write should be auto-flushed. The size reported by FileStatus and the writer should match", - size3, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); + assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", size3, + fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); // Cannot get the current size after closing the log @@ -246,27 +237,24 @@ public class HoodieLogFormatTest { // writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); long size2 = writer.getCurrentSize(); assertTrue("We just wrote a new block - size2 should be > size1", size2 > size1); - assertEquals( - "Write should be auto-flushed. The size reported by FileStatus and the writer should match", - size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen()); + assertEquals("Write should be auto-flushed. The size reported by FileStatus and the writer should match", size2, + fs.getFileStatus(writer.getLogFile().getPath()).getLen()); writer.close(); } @Test - public void testAppendNotSupported() - throws IOException, URISyntaxException, InterruptedException { + public void testAppendNotSupported() throws IOException, URISyntaxException, InterruptedException { // Use some fs like LocalFileSystem, that does not support appends Path localPartitionPath = new Path("file://" + partitionPath); - FileSystem localFs = FSUtils - .getFs(localPartitionPath.toString(), HoodieTestUtils.getDefaultHadoopConf()); + FileSystem localFs = FSUtils.getFs(localPartitionPath.toString(), HoodieTestUtils.getDefaultHadoopConf()); Path testPath = new Path(localPartitionPath, "append_test"); localFs.mkdirs(testPath); @@ -279,8 +267,8 @@ public class HoodieLogFormatTest { for (int i = 0; i < 2; i++) { HoodieLogFormat.newWriterBuilder().onParentPath(testPath) - .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive") - .overBaseCommit("").withFs(localFs).build().appendBlock(dataBlock).close(); + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive").overBaseCommit("") + .withFs(localFs).build().appendBlock(dataBlock).close(); } // ensure there are two log file versions, with same data. @@ -290,16 +278,14 @@ public class HoodieLogFormatTest { @SuppressWarnings("unchecked") @Test - public void testBasicWriteAndScan() - throws IOException, URISyntaxException, InterruptedException { + public void testBasicWriteAndScan() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); Schema schema = getSimpleSchema(); List records = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords = records.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords = records.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); @@ -307,31 +293,27 @@ public class HoodieLogFormatTest { writer = writer.appendBlock(dataBlock); writer.close(); - Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); + Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("We wrote a block, we should be able to read it", reader.hasNext()); HoodieLogBlock nextBlock = reader.next(); - assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK, - nextBlock.getBlockType()); + assertEquals("The next block should be a data block", HoodieLogBlockType.AVRO_DATA_BLOCK, nextBlock.getBlockType()); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords, dataBlockRead.getRecords()); } @SuppressWarnings("unchecked") @Test - public void testBasicAppendAndRead() - throws IOException, URISyntaxException, InterruptedException { + public void testBasicAppendAndRead() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); List records1 = SchemaTestUtil.generateTestRecords(0, 100); Schema schema = getSimpleSchema(); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); @@ -340,12 +322,11 @@ public class HoodieLogFormatTest { writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records2 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords2 = records2.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords2 = records2.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); @@ -353,51 +334,48 @@ public class HoodieLogFormatTest { // Close and Open again and append 100 more records writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records3 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords3 = records3.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords3 = records3.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); - Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); + Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("First block should be available", reader.hasNext()); HoodieLogBlock nextBlock = reader.next(); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) nextBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords1.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords1.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1, dataBlockRead.getRecords()); reader.hasNext(); nextBlock = reader.next(); dataBlockRead = (HoodieAvroDataBlock) nextBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords2.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords2.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords2, dataBlockRead.getRecords()); reader.hasNext(); nextBlock = reader.next(); dataBlockRead = (HoodieAvroDataBlock) nextBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords3.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords3.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords3, dataBlockRead.getRecords()); } @SuppressWarnings("unchecked") @Test - public void testBasicAppendAndScanMultipleFiles() - throws IOException, URISyntaxException, InterruptedException { + public void testBasicAppendAndScanMultipleFiles() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withSizeThreshold(1024).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withSizeThreshold(1024) + .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); @@ -406,12 +384,11 @@ public class HoodieLogFormatTest { Set logFiles = new HashSet<>(); List> allRecords = new ArrayList<>(); // create 4 log files - while(writer.getLogFile().getLogVersion() != 4) { + while (writer.getLogFile().getLogVersion() != 4) { logFiles.add(writer.getLogFile()); List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); allRecords.add(copyOfRecords1); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); @@ -421,24 +398,22 @@ public class HoodieLogFormatTest { // scan all log blocks (across multiple log files) HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), - schema, "100", 10240L, readBlocksLazily, false); + logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), schema, "100", + 10240L, readBlocksLazily, false); List scannedRecords = new ArrayList<>(); - for(HoodieRecord record: scanner) { + for (HoodieRecord record : scanner) { scannedRecords.add((IndexedRecord) record.getData().getInsertValue(schema).get()); } - assertEquals("Scanner records count should be the same as appended records", - scannedRecords.size(), allRecords.stream().flatMap(records -> records.stream()) - .collect(Collectors.toList()).size()); + assertEquals("Scanner records count should be the same as appended records", scannedRecords.size(), + allRecords.stream().flatMap(records -> records.stream()).collect(Collectors.toList()).size()); } @Test - public void testAppendAndReadOnCorruptedLog() - throws IOException, URISyntaxException, InterruptedException { + public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); @@ -467,14 +442,12 @@ public class HoodieLogFormatTest { outputStream.close(); // First round of reads - we should be able to read the first block and then EOF - Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); + Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("First block should be available", reader.hasNext()); reader.next(); assertTrue("We should have corrupted block next", reader.hasNext()); HoodieLogBlock block = reader.next(); - assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, - block.getBlockType()); + assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, block.getBlockType()); HoodieCorruptBlock corruptBlock = (HoodieCorruptBlock) block; //assertEquals("", "something-random", new String(corruptBlock.getCorruptedBytes())); assertFalse("There should be no more block left", reader.hasNext()); @@ -496,8 +469,8 @@ public class HoodieLogFormatTest { // Should be able to append a new block writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); @@ -505,16 +478,14 @@ public class HoodieLogFormatTest { writer.close(); // Second round of reads - we should be able to read the first and last block - reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); + reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); assertTrue("First block should be available", reader.hasNext()); reader.next(); assertTrue("We should get the 1st corrupted block next", reader.hasNext()); reader.next(); assertTrue("We should get the 2nd corrupted block next", reader.hasNext()); block = reader.next(); - assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, - block.getBlockType()); + assertEquals("The read block should be a corrupt block", HoodieLogBlockType.CORRUPT_BLOCK, block.getBlockType()); corruptBlock = (HoodieCorruptBlock) block; //assertEquals("", "something-else-random", new String(corruptBlock.getCorruptedBytes())); assertTrue("We should get the last block next", reader.hasNext()); @@ -524,8 +495,7 @@ public class HoodieLogFormatTest { @Test - public void testAvroLogRecordReaderBasic() - throws IOException, URISyntaxException, InterruptedException { + public void testAvroLogRecordReaderBasic() throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) @@ -533,9 +503,8 @@ public class HoodieLogFormatTest { .overBaseCommit("100").withFs(fs).withSizeThreshold(500).build(); // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); @@ -545,22 +514,18 @@ public class HoodieLogFormatTest { // Write 2 List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords2 = records2.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords2 = records2.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); writer.close(); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, - schema, "100", 10240L, readBlocksLazily, false); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "100", 10240L, readBlocksLazily, false); assertEquals("", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -568,10 +533,8 @@ public class HoodieLogFormatTest { copyOfRecords1.addAll(copyOfRecords2); Set originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toSet()); - assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, - readKeys); + .collect(Collectors.toSet()); + assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, readKeys); } @Test @@ -585,9 +548,8 @@ public class HoodieLogFormatTest { // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); @@ -612,34 +574,27 @@ public class HoodieLogFormatTest { // Write 3 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords3 = records3.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords3 = records3.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, - schema, "102", 10240L, readBlocksLazily, false); - assertEquals("We read 200 records from 2 write batches", 200, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "102", 10240L, readBlocksLazily, false); + assertEquals("We read 200 records from 2 write batches", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); copyOfRecords1.addAll(copyOfRecords3); Set originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toSet()); - assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, - readKeys); + .collect(Collectors.toSet()); + assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, readKeys); } @Test @@ -653,9 +608,8 @@ public class HoodieLogFormatTest { // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -690,42 +644,35 @@ public class HoodieLogFormatTest { String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); writer = writer.appendBlock(commandBlock); // Write 3 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103"); List records3 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords3 = records3.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords3 = records3.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, - schema, "103", 10240L, true, false); - assertEquals("We would read 200 records", 200, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "103", 10240L, true, false); + assertEquals("We would read 200 records", 200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 200 records", 200, readKeys.size()); copyOfRecords1.addAll(copyOfRecords3); Set originalKeys = copyOfRecords1.stream() .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toSet()); - assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, - readKeys); + .collect(Collectors.toSet()); + assertEquals("CompositeAvroLogReader should return 200 records from 2 versions", originalKeys, readKeys); } @Test @@ -739,9 +686,8 @@ public class HoodieLogFormatTest { // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -751,44 +697,35 @@ public class HoodieLogFormatTest { // Write 2 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords2 = records2.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords2 = records2.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); copyOfRecords1.addAll(copyOfRecords2); - List originalKeys = copyOfRecords1.stream() - .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toList()); + List originalKeys = copyOfRecords1.stream().map( + s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - header); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); writer = writer.appendBlock(deleteBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, - schema, "102", 10240L, readBlocksLazily, false); - assertEquals("We still would read 200 records", 200, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "102", 10240L, readBlocksLazily, false); + assertEquals("We still would read 200 records", 200, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 150 records", 150, readKeys.size()); originalKeys.removeAll(deletedKeys); Collections.sort(originalKeys); Collections.sort(readKeys); - assertEquals("CompositeAvroLogReader should return 150 records from 2 versions", originalKeys, - readKeys); + assertEquals("CompositeAvroLogReader should return 150 records from 2 versions", originalKeys, readKeys); // Rollback the last block header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103"); @@ -802,8 +739,7 @@ public class HoodieLogFormatTest { scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, false); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); - assertEquals("Stream collect should return all 200 records after rollback of delete", 200, - readKeys.size()); + assertEquals("Stream collect should return all 200 records after rollback of delete", 200, readKeys.size()); } @Test @@ -819,8 +755,8 @@ public class HoodieLogFormatTest { // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); @@ -835,15 +771,12 @@ public class HoodieLogFormatTest { dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); - List originalKeys = copyOfRecords1.stream() - .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toList()); + List originalKeys = copyOfRecords1.stream().map( + s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - header); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); writer = writer.appendBlock(deleteBlock); // Attempt 1 : Write rollback block for a failed write @@ -860,14 +793,12 @@ public class HoodieLogFormatTest { // Attempt 2 : Write another rollback blocks for a failed write writer = writer.appendBlock(commandBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); // all data must be rolled back before merge - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100", 10240L, readBlocksLazily, false); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "100", 10240L, readBlocksLazily, false); assertEquals("We would have scanned 0 records because of rollback", 0, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(); @@ -888,9 +819,8 @@ public class HoodieLogFormatTest { // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); @@ -898,15 +828,12 @@ public class HoodieLogFormatTest { HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); writer = writer.appendBlock(dataBlock); - List originalKeys = copyOfRecords1.stream() - .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toList()); + List originalKeys = copyOfRecords1.stream().map( + s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - header); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); writer = writer.appendBlock(deleteBlock); // Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write @@ -916,15 +843,12 @@ public class HoodieLogFormatTest { writer = writer.appendBlock(commandBlock); writer = writer.appendBlock(commandBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100", 10240L, readBlocksLazily, false); - assertEquals("We would read 0 records", 0, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "100", 10240L, readBlocksLazily, false); + assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @Test @@ -951,15 +875,12 @@ public class HoodieLogFormatTest { HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "100", 10240L, readBlocksLazily, false); - assertEquals("We still would read 100 records", 100, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "100", 10240L, readBlocksLazily, false); + assertEquals("We still would read 100 records", 100, scanner.getTotalLogRecords()); final List readKeys = new ArrayList<>(100); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals("Stream collect should return all 150 records", 100, readKeys.size()); @@ -978,9 +899,8 @@ public class HoodieLogFormatTest { // Write 1 List records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); @@ -990,15 +910,12 @@ public class HoodieLogFormatTest { writer = writer.appendBlock(dataBlock); writer = writer.appendBlock(dataBlock); - List originalKeys = copyOfRecords1.stream() - .map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) - .collect( - Collectors.toList()); + List originalKeys = copyOfRecords1.stream().map( + s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()).collect(Collectors.toList()); // Delete 50 keys List deletedKeys = originalKeys.subList(0, 50); - HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), - header); + HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new String[50]), header); writer = writer.appendBlock(deleteBlock); // Write 1 rollback block for a failed write @@ -1008,15 +925,12 @@ public class HoodieLogFormatTest { HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "101", 10240L, readBlocksLazily, false); - assertEquals("We would read 0 records", 0, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "101", 10240L, readBlocksLazily, false); + assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @Test @@ -1069,8 +983,8 @@ public class HoodieLogFormatTest { outputStream.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); writer = writer.appendBlock(dataBlock); writer.close(); @@ -1089,8 +1003,8 @@ public class HoodieLogFormatTest { outputStream.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); // Write 1 rollback block for a failed write header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, @@ -1098,15 +1012,12 @@ public class HoodieLogFormatTest { HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer = writer.appendBlock(commandBlock); - List allLogFiles = FSUtils - .getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()) - .collect(Collectors.toList()); + List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, + "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, - allLogFiles, schema, "101", 10240L, readBlocksLazily, false); - assertEquals("We would read 0 records", 0, - scanner.getTotalLogRecords()); + HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, basePath, allLogFiles, schema, + "101", 10240L, readBlocksLazily, false); + assertEquals("We would read 0 records", 0, scanner.getTotalLogRecords()); } @Test @@ -1133,7 +1044,7 @@ public class HoodieLogFormatTest { records = SchemaTestUtil.generateHoodieTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, new String(HoodieAvroUtils.compress(schema.toString()))); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, schema); - byte [] content = dataBlock.getBytes(schema); + byte[] content = dataBlock.getBytes(schema); outputStream.writeInt(content.length); // Write out some content outputStream.write(content); @@ -1142,8 +1053,8 @@ public class HoodieLogFormatTest { outputStream.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); // Write 2 with MAGIC and latest log format version records = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -1153,16 +1064,15 @@ public class HoodieLogFormatTest { // Write 3 with MAGIC and latest log format version writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateHoodieTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); writer.close(); - Reader reader = HoodieLogFormat - .newReader(fs, writer.getLogFile(), schema); + Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), schema); // Read the first block written with latest version and magic reader.hasNext(); @@ -1189,16 +1099,14 @@ public class HoodieLogFormatTest { @SuppressWarnings("unchecked") @Test - public void testBasicAppendAndReadInReverse() - throws IOException, URISyntaxException, InterruptedException { + public void testBasicAppendAndReadInReverse() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); Schema schema = getSimpleSchema(); List records1 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -1207,54 +1115,51 @@ public class HoodieLogFormatTest { writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records2 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords2 = records2.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords2 = records2.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); writer.close(); // Close and Open again and append 100 more records writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records3 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords3 = records3.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords3 = records3.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); - HoodieLogFileReader reader = - new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), readBlocksLazily, - true); + HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), + readBlocksLazily, true); assertTrue("Last block should be available", reader.hasPrev()); HoodieLogBlock prevBlock = reader.prev(); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) prevBlock; - assertEquals("Third records size should be equal to the written records size", - copyOfRecords3.size(), dataBlockRead.getRecords().size()); + assertEquals("Third records size should be equal to the written records size", copyOfRecords3.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords3, dataBlockRead.getRecords()); assertTrue("Second block should be available", reader.hasPrev()); prevBlock = reader.prev(); dataBlockRead = (HoodieAvroDataBlock) prevBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords2.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords2.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords2, dataBlockRead.getRecords()); assertTrue("First block should be available", reader.hasPrev()); prevBlock = reader.prev(); dataBlockRead = (HoodieAvroDataBlock) prevBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords1.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords1.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1, dataBlockRead.getRecords()); @@ -1262,8 +1167,7 @@ public class HoodieLogFormatTest { } @Test - public void testAppendAndReadOnCorruptedLogInReverse() - throws IOException, URISyntaxException, InterruptedException { + public void testAppendAndReadOnCorruptedLogInReverse() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); @@ -1295,16 +1199,15 @@ public class HoodieLogFormatTest { // Should be able to append a new block writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); dataBlock = new HoodieAvroDataBlock(records, header); writer = writer.appendBlock(dataBlock); writer.close(); // First round of reads - we should be able to read the first block and then EOF - HoodieLogFileReader reader = - new HoodieLogFileReader(fs, writer.getLogFile(), schema, readBlocksLazily, true); + HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), schema, readBlocksLazily, true); assertTrue("Last block should be available", reader.hasPrev()); HoodieLogBlock block = reader.prev(); @@ -1313,7 +1216,7 @@ public class HoodieLogFormatTest { assertTrue("Last block should be available", reader.hasPrev()); try { reader.prev(); - } catch(CorruptedLogFileException e) { + } catch (CorruptedLogFileException e) { e.printStackTrace(); // We should have corrupted block } @@ -1321,16 +1224,14 @@ public class HoodieLogFormatTest { @SuppressWarnings("unchecked") @Test - public void testBasicAppendAndTraverseInReverse() - throws IOException, URISyntaxException, InterruptedException { + public void testBasicAppendAndTraverseInReverse() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") .overBaseCommit("100").withFs(fs).build(); Schema schema = getSimpleSchema(); List records1 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords1 = records1.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords1 = records1.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); @@ -1339,31 +1240,28 @@ public class HoodieLogFormatTest { writer.close(); writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records2 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords2 = records2.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords2 = records2.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records2, header); writer = writer.appendBlock(dataBlock); writer.close(); // Close and Open again and append 100 more records writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1") - .overBaseCommit("100").withFs(fs).build(); + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1").overBaseCommit("100") + .withFs(fs).build(); List records3 = SchemaTestUtil.generateTestRecords(0, 100); - List copyOfRecords3 = records3.stream().map(record -> - HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)) - .collect(Collectors.toList()); + List copyOfRecords3 = records3.stream().map( + record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = new HoodieAvroDataBlock(records3, header); writer = writer.appendBlock(dataBlock); writer.close(); - HoodieLogFileReader reader = - new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), readBlocksLazily, - true); + HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), + readBlocksLazily, true); assertTrue("Third block should be available", reader.hasPrev()); reader.moveToPrev(); @@ -1375,8 +1273,8 @@ public class HoodieLogFormatTest { assertTrue("First block should be available", reader.hasPrev()); HoodieLogBlock prevBlock = reader.prev(); HoodieAvroDataBlock dataBlockRead = (HoodieAvroDataBlock) prevBlock; - assertEquals("Read records size should be equal to the written records size", - copyOfRecords1.size(), dataBlockRead.getRecords().size()); + assertEquals("Read records size should be equal to the written records size", copyOfRecords1.size(), + dataBlockRead.getRecords().size()); assertEquals("Both records lists should be the same. (ordering guaranteed)", copyOfRecords1, dataBlockRead.getRecords()); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java index 096c75d77..25a6a00f4 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java @@ -54,25 +54,16 @@ public class HoodieActiveTimelineTest { @Test public void testLoadingInstantsFromFiles() throws IOException { - HoodieInstant instant1 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant instant2 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "3"); - HoodieInstant instant3 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5"); - HoodieInstant instant4 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "8"); - HoodieInstant instant1_complete = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant instant2_complete = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); - HoodieInstant instant3_complete = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5"); - HoodieInstant instant4_complete = - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "8"); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant3 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5"); + HoodieInstant instant4 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "8"); + HoodieInstant instant1Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant2Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant3Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5"); + HoodieInstant instant4Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "8"); - HoodieInstant instant5 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9"); + HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9"); timeline = new HoodieActiveTimeline(metaClient); timeline.saveAsComplete(instant1, Optional.empty()); @@ -83,14 +74,14 @@ public class HoodieActiveTimelineTest { timeline = timeline.reload(); assertEquals("Total instants should be 5", 5, timeline.countInstants()); - HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream - .of(instant1_complete, instant2_complete, instant3_complete, instant4_complete, - instant5), timeline.getInstants()); - HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream - .of(instant1_complete, instant2_complete, instant3_complete, instant4_complete, - instant5), timeline.getCommitTimeline().getInstants()); HoodieTestUtils.assertStreamEquals("Check the instants stream", - Stream.of(instant1_complete, instant2_complete, instant3_complete, instant4_complete), + Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete, instant5), + timeline.getInstants()); + HoodieTestUtils.assertStreamEquals("Check the instants stream", + Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete, instant5), + timeline.getCommitTimeline().getInstants()); + HoodieTestUtils.assertStreamEquals("Check the instants stream", + Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete), timeline.getCommitTimeline().filterCompletedInstants().getInstants()); HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream.of(instant5), timeline.getCommitTimeline().filterInflights().getInstants()); @@ -105,21 +96,19 @@ public class HoodieActiveTimelineTest { assertEquals("", Optional.empty(), timeline.nthInstant(5)); assertEquals("", Optional.empty(), timeline.nthInstant(-1)); assertEquals("", Optional.empty(), timeline.lastInstant()); - assertFalse("", timeline.containsInstant( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01"))); + assertFalse("", timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01"))); } @Test public void testTimelineOperations() throws Exception { - timeline = new MockHoodieTimeline( - Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"), + timeline = new MockHoodieTimeline(Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"), Stream.of("21", "23")); HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"), - timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11") - .getInstants().map(HoodieInstant::getTimestamp)); + timeline.getCommitTimeline().filterCompletedInstants().findInstantsInRange("04", "11").getInstants() + .map(HoodieInstant::getTimestamp)); HoodieTestUtils.assertStreamEquals("", Stream.of("09", "11"), - timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2) - .getInstants().map(HoodieInstant::getTimestamp)); + timeline.getCommitTimeline().filterCompletedInstants().findInstantsAfter("07", 2).getInstants() + .map(HoodieInstant::getTimestamp)); assertFalse(timeline.empty()); assertFalse(timeline.getCommitTimeline().filterInflights().empty()); assertEquals("", 12, timeline.countInstants()); @@ -130,8 +119,7 @@ public class HoodieActiveTimelineTest { assertEquals("", "11", activeCommitTimeline.nthInstant(5).get().getTimestamp()); assertEquals("", "19", activeCommitTimeline.lastInstant().get().getTimestamp()); assertEquals("", "09", activeCommitTimeline.nthFromLastInstant(5).get().getTimestamp()); - assertTrue("", activeCommitTimeline.containsInstant( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09"))); + assertTrue("", activeCommitTimeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09"))); assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02")); assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00")); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java index 5e3b48844..83360e416 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java @@ -27,13 +27,10 @@ import java.util.stream.Stream; public class MockHoodieTimeline extends HoodieActiveTimeline { - public MockHoodieTimeline(Stream completed, Stream inflights) - throws IOException { + public MockHoodieTimeline(Stream completed, Stream inflights) throws IOException { super(); - this.instants = Stream.concat(completed - .map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)), - inflights.map( - s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s))) + this.instants = Stream.concat(completed.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)), + inflights.map(s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s))) .sorted(Comparator.comparing(new Function() { @Override public String apply(HoodieInstant hoodieInstant) { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index fab8e3714..f26f6d3cd 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -69,8 +69,7 @@ public class HoodieTableFileSystemViewTest { metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); if (statuses != null) { fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), - statuses); + metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), statuses); } else { fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants()); @@ -86,8 +85,8 @@ public class HoodieTableFileSystemViewTest { String fileId = UUID.randomUUID().toString(); assertFalse("No commit, should not find any data file", - roView.getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().isPresent()); + roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst() + .isPresent()); // Only one commit, but is not safe String commitTime1 = "1"; @@ -95,43 +94,34 @@ public class HoodieTableFileSystemViewTest { new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); refreshFsView(null); assertFalse("No commit, should not find any data file", - roView.getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)) - .findFirst().isPresent()); + roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst() + .isPresent()); // Make this commit safe HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); commitTimeline.saveAsComplete(instant1, Optional.empty()); refreshFsView(null); - assertEquals("", fileName1, roView - .getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)) - .findFirst().get() - .getFileName()); + assertEquals("", fileName1, + roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get() + .getFileName()); // Do another commit, but not safe String commitTime2 = "2"; String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId); new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); refreshFsView(null); - assertEquals("", fileName1, roView - .getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)) - .findFirst().get() - .getFileName()); + assertEquals("", fileName1, + roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get() + .getFileName()); // Make it safe - HoodieInstant instant2 = - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2); + HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2); commitTimeline.saveAsComplete(instant2, Optional.empty()); refreshFsView(null); - assertEquals("", fileName2, roView - .getLatestDataFiles(partitionPath) - .filter(dfile -> dfile.getFileId().equals(fileId)) - .findFirst().get() - .getFileName()); + assertEquals("", fileName2, + roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get() + .getFileName()); } @Test @@ -148,31 +138,20 @@ public class HoodieTableFileSystemViewTest { String fileId3 = UUID.randomUUID().toString(); String fileId4 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)) + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1)) .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) .createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); @@ -188,16 +167,15 @@ public class HoodieTableFileSystemViewTest { // Check files as of lastest commit. List allSlices = rtView.getAllFileSlices("2016/05/01").collect(Collectors.toList()); assertEquals(8, allSlices.size()); - Map fileSliceMap = allSlices.stream().collect(Collectors.groupingBy( - slice -> slice.getFileId(), Collectors.counting())); + Map fileSliceMap = allSlices.stream().collect( + Collectors.groupingBy(slice -> slice.getFileId(), Collectors.counting())); assertEquals(2, fileSliceMap.get(fileId1).longValue()); assertEquals(3, fileSliceMap.get(fileId2).longValue()); assertEquals(2, fileSliceMap.get(fileId3).longValue()); assertEquals(1, fileSliceMap.get(fileId4).longValue()); - List dataFileList = - roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4) - .collect(Collectors.toList()); + List dataFileList = roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4) + .collect(Collectors.toList()); assertEquals(3, dataFileList.size()); Set filenames = Sets.newHashSet(); for (HoodieDataFile status : dataFileList) { @@ -208,28 +186,21 @@ public class HoodieTableFileSystemViewTest { assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3))); filenames = Sets.newHashSet(); - List logFilesList = - rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4) - .map(slice -> slice.getLogFiles()) - .flatMap(logFileList -> logFileList) - .collect(Collectors.toList()); + List logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4) + .map(slice -> slice.getLogFiles()).flatMap(logFileList -> logFileList) + .collect(Collectors.toList()); assertEquals(logFilesList.size(), 4); for (HoodieLogFile logFile : logFilesList) { filenames.add(logFile.getFileName()); } - assertTrue(filenames - .contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))); - assertTrue(filenames - .contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1))); - assertTrue(filenames - .contains(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))); - assertTrue(filenames - .contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))); + assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))); + assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1))); + assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))); + assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))); // Reset the max commit time - List dataFiles = - roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3) - .collect(Collectors.toList()); + List dataFiles = roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3) + .collect(Collectors.toList()); assertEquals(dataFiles.size(), 3); filenames = Sets.newHashSet(); for (HoodieDataFile status : dataFiles) { @@ -239,10 +210,8 @@ public class HoodieTableFileSystemViewTest { assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2))); assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3))); - logFilesList = - rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3) - .map(slice -> slice.getLogFiles()) - .flatMap(logFileList -> logFileList).collect(Collectors.toList()); + logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3).map(slice -> slice.getLogFiles()) + .flatMap(logFileList -> logFileList).collect(Collectors.toList()); assertEquals(logFilesList.size(), 1); assertTrue(logFilesList.get(0).getFileName() .equals(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))); @@ -261,20 +230,13 @@ public class HoodieTableFileSystemViewTest { String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)) - .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -286,8 +248,7 @@ public class HoodieTableFileSystemViewTest { assertEquals(7, statuses.length); refreshFsView(null); - List fileGroups = - fsView.getAllFileGroups("2016/05/01").collect(Collectors.toList()); + List fileGroups = fsView.getAllFileGroups("2016/05/01").collect(Collectors.toList()); assertEquals(3, fileGroups.size()); for (HoodieFileGroup fileGroup : fileGroups) { @@ -298,18 +259,14 @@ public class HoodieTableFileSystemViewTest { filenames.add(dataFile.getFileName()); }); if (fileId.equals(fileId1)) { - assertEquals(filenames, - Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId1), - FSUtils.makeDataFileName(commitTime4, 1, fileId1))); + assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId1), + FSUtils.makeDataFileName(commitTime4, 1, fileId1))); } else if (fileId.equals(fileId2)) { - assertEquals(filenames, - Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId2), - FSUtils.makeDataFileName(commitTime2, 1, fileId2), - FSUtils.makeDataFileName(commitTime3, 1, fileId2))); + assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId2), + FSUtils.makeDataFileName(commitTime2, 1, fileId2), FSUtils.makeDataFileName(commitTime3, 1, fileId2))); } else { - assertEquals(filenames, - Sets.newHashSet(FSUtils.makeDataFileName(commitTime3, 1, fileId3), - FSUtils.makeDataFileName(commitTime4, 1, fileId3))); + assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime3, 1, fileId3), + FSUtils.makeDataFileName(commitTime4, 1, fileId3))); } } } @@ -327,28 +284,19 @@ public class HoodieTableFileSystemViewTest { String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId1)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId1)).createNewFile(); + + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) - .createNewFile(); - - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)) - .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -360,8 +308,7 @@ public class HoodieTableFileSystemViewTest { assertEquals(9, statuses.length); refreshFsView(statuses); - List dataFiles = roView - .getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3)) + List dataFiles = roView.getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3)) .collect(Collectors.toList()); assertEquals(3, dataFiles.size()); Set filenames = Sets.newHashSet(); @@ -371,8 +318,7 @@ public class HoodieTableFileSystemViewTest { assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2))); assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3))); - List slices = rtView - .getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4)) + List slices = rtView.getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4)) .collect(Collectors.toList()); assertEquals(3, slices.size()); for (FileSlice slice : slices) { @@ -406,20 +352,13 @@ public class HoodieTableFileSystemViewTest { String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)) - .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -431,9 +370,8 @@ public class HoodieTableFileSystemViewTest { assertEquals(7, statuses.length); refreshFsView(null); - List dataFiles = - roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2) - .collect(Collectors.toList()); + List dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2) + .collect(Collectors.toList()); assertEquals(2, dataFiles.size()); Set filenames = Sets.newHashSet(); for (HoodieDataFile status : dataFiles) { @@ -457,31 +395,21 @@ public class HoodieTableFileSystemViewTest { String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)) - .createNewFile(); - new File(fullPartitionPath + FSUtils - .makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)) + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0)) .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)) - .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)) - .createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -494,9 +422,7 @@ public class HoodieTableFileSystemViewTest { refreshFsView(statuses); - List fileGroups = fsView - .getAllFileGroups(partitionPath) - .collect(Collectors.toList()); + List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); assertEquals(3, fileGroups.size()); for (HoodieFileGroup fileGroup : fileGroups) { List slices = fileGroup.getAllFileSlices().collect(Collectors.toList()); @@ -516,8 +442,7 @@ public class HoodieTableFileSystemViewTest { } } - List statuses1 = - roView.getLatestDataFiles().collect(Collectors.toList()); + List statuses1 = roView.getLatestDataFiles().collect(Collectors.toList()); assertEquals(3, statuses1.size()); Set filenames = Sets.newHashSet(); for (HoodieDataFile status : statuses1) { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java index a736da8e5..02c8933cb 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java @@ -20,10 +20,8 @@ import com.uber.hoodie.avro.MercifulJsonConverter; import com.uber.hoodie.common.model.HoodieAvroPayload; 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.timeline.HoodieActiveTimeline; import com.uber.hoodie.exception.HoodieIOException; - import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -40,9 +38,7 @@ import java.util.Optional; import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; - import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -51,19 +47,16 @@ import org.apache.avro.io.DecoderFactory; public class SchemaTestUtil { public static Schema getSimpleSchema() throws IOException { - return new Schema.Parser() - .parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avro")); + return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avro")); } - public static List generateTestRecords(int from, int limit) - throws IOException, URISyntaxException { + public static List generateTestRecords(int from, int limit) throws IOException, URISyntaxException { return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit); } - private static List toRecords(Schema writerSchema, Schema readerSchema, int from, - int limit) throws IOException, URISyntaxException { - GenericDatumReader reader = - new GenericDatumReader<>(writerSchema, readerSchema); + private static List toRecords(Schema writerSchema, Schema readerSchema, int from, int limit) + throws IOException, URISyntaxException { + GenericDatumReader reader = new GenericDatumReader<>(writerSchema, readerSchema); // Required to register the necessary JAR:// file system URI resource = SchemaTestUtil.class.getClass().getResource("/sample.data").toURI(); Path dataPath; @@ -103,15 +96,12 @@ public class SchemaTestUtil { List records = generateTestRecords(from, limit); String commitTime = HoodieActiveTimeline.createNewCommitTime(); Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); - return records.stream() - .map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema)) - .map(p -> { - p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString()); - p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00"); - p.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime); - return p; - }).collect( - Collectors.toList()); + return records.stream().map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema)).map(p -> { + p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString()); + p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00"); + p.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime); + return p; + }).collect(Collectors.toList()); } @@ -131,8 +121,7 @@ public class SchemaTestUtil { } public static List updateHoodieTestRecords(List oldRecordKeys, List newRecords, - String commitTime) - throws IOException, URISyntaxException { + String commitTime) throws IOException, URISyntaxException { return newRecords.stream() .map(p -> { @@ -155,8 +144,9 @@ public class SchemaTestUtil { new HoodieAvroPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList()); } - public static List updateHoodieTestRecordsWithoutHoodieMetadata(List oldRecords, Schema schema, - String fieldNameToUpdate, String newValue) + public static List updateHoodieTestRecordsWithoutHoodieMetadata(List oldRecords, + Schema schema, + String fieldNameToUpdate, String newValue) throws IOException, URISyntaxException { return oldRecords .stream() @@ -173,8 +163,7 @@ public class SchemaTestUtil { } public static Schema getEvolvedSchema() throws IOException { - return new Schema.Parser() - .parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro")); + return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro")); } public static List generateEvolvedTestRecords(int from, int limit) @@ -183,12 +172,11 @@ public class SchemaTestUtil { } public static Schema getComplexEvolvedSchema() throws IOException { - return new Schema.Parser() - .parse(SchemaTestUtil.class.getResourceAsStream("/complex-test-evolved.avro")); + return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/complex-test-evolved.avro")); } - public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber, - String commitTime, String fileId) throws IOException { + public static GenericRecord generateAvroRecordFromJson(Schema schema, int recordNumber, String commitTime, + String fileId) throws IOException { TestRecord record = new TestRecord(commitTime, recordNumber, fileId); MercifulJsonConverter converter = new MercifulJsonConverter(schema); return converter.convert(record.toJsonString()); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java index f8360376e..42549afcf 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java @@ -20,18 +20,17 @@ import com.uber.hoodie.common.model.HoodieAvroPayload; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; - import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; public class SpillableMapTestUtils { public static List upsertRecords(List iRecords, - Map> records) { + Map> records) { List recordKeys = new ArrayList<>(); iRecords .stream() diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java index 3e05158b2..fff22c7e1 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java @@ -31,8 +31,7 @@ import org.junit.contrib.java.lang.system.EnvironmentVariables; public class TestFSUtils { @Rule - public final EnvironmentVariables environmentVariables - = new EnvironmentVariables(); + public final EnvironmentVariables environmentVariables = new EnvironmentVariables(); @Test public void testMakeDataFileName() { @@ -51,8 +50,10 @@ public class TestFSUtils { int stageId = Integer.MAX_VALUE; long taskAttemptId = Long.MAX_VALUE; String fileName = UUID.randomUUID().toString(); - assertTrue(FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId, taskAttemptId) - .equals(partitionPath.replace("/", "-") + "_" + fileName + "_" + taskPartitionId + "_" + commitTime + "_" + stageId + "_" + taskAttemptId + ".parquet")); + assertTrue( + FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId, taskAttemptId) + .equals(partitionPath.replace("/", "-") + "_" + fileName + "_" + taskPartitionId + "_" + commitTime + "_" + + stageId + "_" + taskAttemptId + ".parquet")); } @Test diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java index fd0a4475a..636df99e3 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java @@ -63,13 +63,12 @@ public class TestParquetUtils { // Write out a parquet file Schema schema = HoodieAvroUtils.getRecordKeySchema(); BloomFilter filter = new BloomFilter(1000, 0.0001); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( - new AvroSchemaConverter().convert(schema), schema, filter); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, + filter); String filePath = basePath + "/test.parquet"; - ParquetWriter writer = new ParquetWriter(new Path(filePath), - writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, - ParquetWriter.DEFAULT_PAGE_SIZE); + ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, + 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE); for (String rowKey : rowKeys) { GenericRecord rec = new GenericData.Record(schema); rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey); @@ -80,15 +79,13 @@ public class TestParquetUtils { // Read and verify List rowKeysInFile = new ArrayList<>( - ParquetUtils - .readRowKeysFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath))); + ParquetUtils.readRowKeysFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath))); Collections.sort(rowKeysInFile); Collections.sort(rowKeys); assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile); - BloomFilter filterInFile = ParquetUtils - .readBloomFilterFromParquetMetadata(HoodieTestUtils.getDefaultHadoopConf(), - new Path(filePath)); + BloomFilter filterInFile = ParquetUtils.readBloomFilterFromParquetMetadata(HoodieTestUtils.getDefaultHadoopConf(), + new Path(filePath)); for (String rowKey : rowKeys) { assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey)); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java index 5eee98618..ee2c77b5e 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java @@ -20,18 +20,18 @@ import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.uber.hoodie.common.model.AvroBinaryTestPayload; import com.uber.hoodie.common.model.HoodieAvroPayload; 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.AvroBinaryTestPayload; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.common.util.SpillableMapTestUtils; import com.uber.hoodie.common.util.SpillableMapUtils; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; +import com.uber.hoodie.common.util.collection.converter.StringConverter; import java.io.IOException; import java.io.UncheckedIOException; import java.net.URISyntaxException; @@ -46,6 +46,7 @@ import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; +import org.junit.Ignore; import org.junit.Test; public class TestDiskBasedMap { @@ -58,8 +59,7 @@ public class TestDiskBasedMap { DiskBasedMap records = new DiskBasedMap<>(Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); - String commitTime = ((GenericRecord) iRecords.get(0)) - .get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + ((GenericRecord) iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); // make sure records have spilled to disk @@ -108,8 +108,7 @@ public class TestDiskBasedMap { DiskBasedMap records = new DiskBasedMap<>(Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, payloadClazz)); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); - String commitTime = ((GenericRecord) iRecords.get(0)) - .get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + // perform some inserts List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -125,9 +124,6 @@ public class TestDiskBasedMap { String newCommitTime = ((GenericRecord) updatedRecords.get(0)) .get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); - // new commit time should be different - assertEquals(commitTime, newCommitTime); - // perform upserts recordKeys = SpillableMapTestUtils.upsertRecords(updatedRecords, records); @@ -194,6 +190,12 @@ public class TestDiskBasedMap { assertTrue(payloadSize > 0); } + /** + * vb - Disabled this test after talking to Nishanth as this relies on timing and sometimes fails in my laptop. + * This specific test sometime takes more than 100 ms (In one case, saw 122 ms), + * @na: TODO: Please check if this can be removed. + */ + @Ignore @Test public void testSizeEstimatorPerformance() throws IOException, URISyntaxException { // Test sizeEstimatorPerformance with simpleSchema @@ -205,6 +207,7 @@ public class TestDiskBasedMap { long startTime = System.currentTimeMillis(); SpillableMapUtils.computePayloadSize(record, converter); long timeTaken = System.currentTimeMillis() - startTime; + System.out.println("Time taken :" + timeTaken); assertTrue(timeTaken < 100); } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java index 1c56b6cb8..c62aa1cc4 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -28,8 +28,8 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.common.util.SpillableMapTestUtils; -import com.uber.hoodie.common.util.collection.converter.StringConverter; import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter; +import com.uber.hoodie.common.util.collection.converter.StringConverter; import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; @@ -55,9 +55,8 @@ public class TestExternalSpillableMap { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.empty(), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.empty(), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -78,9 +77,8 @@ public class TestExternalSpillableMap { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -90,9 +88,8 @@ public class TestExternalSpillableMap { HoodieRecord rec = itr.next(); assert recordKeys.contains(rec.getRecordKey()); } - List updatedRecords = - SchemaTestUtil.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100), - HoodieActiveTimeline.createNewCommitTime()); + List updatedRecords = SchemaTestUtil.updateHoodieTestRecords(recordKeys, + SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewCommitTime()); // update records already inserted SpillableMapTestUtils.upsertRecords(updatedRecords, records); @@ -118,9 +115,8 @@ public class TestExternalSpillableMap { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.empty(), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.empty(), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); // insert a bunch of records so that values spill to disk too @@ -174,9 +170,8 @@ public class TestExternalSpillableMap { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -200,9 +195,8 @@ public class TestExternalSpillableMap { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk @@ -221,15 +215,14 @@ public class TestExternalSpillableMap { List keysToBeUpdated = new ArrayList<>(); keysToBeUpdated.add(key); // Update the commitTime for this record - List updatedRecords = - SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime); + List updatedRecords = SchemaTestUtil + .updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime); // Upsert this updated record SpillableMapTestUtils.upsertRecords(updatedRecords, records); GenericRecord gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); // The record returned for this key should have the updated commitTime assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); - // Get a record from the disk based map key = recordKeys.get(recordKeys.size() - 1); record = records.get(key); @@ -240,8 +233,7 @@ public class TestExternalSpillableMap { keysToBeUpdated = new ArrayList<>(); keysToBeUpdated.add(key); // Update the commitTime for this record - updatedRecords = - SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime); + updatedRecords = SchemaTestUtil.updateHoodieTestRecords(keysToBeUpdated, recordsToUpdate, newCommitTime); // Upsert this updated record SpillableMapTestUtils.upsertRecords(updatedRecords, records); gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get(); @@ -256,9 +248,8 @@ public class TestExternalSpillableMap { String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<> - (16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), - new HoodieRecordConverter(schema, payloadClazz)); //16B + new ExternalSpillableMap<>(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(), + new HoodieRecordConverter(schema, payloadClazz)); //16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieHiveUtil.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieHiveUtil.java index 12b4abf40..70acdf73b 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieHiveUtil.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieHiveUtil.java @@ -23,8 +23,7 @@ import org.apache.log4j.Logger; public class HoodieHiveUtil { - public static final Logger LOG = - LogManager.getLogger(HoodieHiveUtil.class); + public static final Logger LOG = LogManager.getLogger(HoodieHiveUtil.class); public static final String HOODIE_CONSUME_MODE_PATTERN = "hoodie.%s.consume.mode"; public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp"; diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java index 302373e1b..84d7da39c 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java @@ -71,8 +71,7 @@ import parquet.io.api.Binary; * Hoodie/Non-Hoodie datasets */ @UseFileSplitsFromInputFormat -public class HoodieInputFormat extends MapredParquetInputFormat - implements Configurable { +public class HoodieInputFormat extends MapredParquetInputFormat implements Configurable { public static final Log LOG = LogFactory.getLog(HoodieInputFormat.class); @@ -99,7 +98,8 @@ public class HoodieInputFormat extends MapredParquetInputFormat } String tableName = metadata.getTableConfig().getTableName(); String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName); - // Get all commits, delta commits, compactions, as all of them produce a base parquet file today + // Get all commits, delta commits, compactions, as all of them produce a base parquet file + // today HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants(); TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metadata, @@ -112,19 +112,16 @@ public class HoodieInputFormat extends MapredParquetInputFormat // Total number of commits to return in this batch. Set this to -1 to get all the commits. Integer maxCommits = HoodieHiveUtil.readMaxCommits(Job.getInstance(job), tableName); LOG.info("Last Incremental timestamp was set as " + lastIncrementalTs); - List commitsToReturn = - timeline.findInstantsAfter(lastIncrementalTs, maxCommits).getInstants() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); - List filteredFiles = roView - .getLatestDataFilesInRange(commitsToReturn) + List commitsToReturn = timeline.findInstantsAfter(lastIncrementalTs, maxCommits) + .getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + List filteredFiles = roView.getLatestDataFilesInRange(commitsToReturn) .collect(Collectors.toList()); for (HoodieDataFile filteredFile : filteredFiles) { LOG.info("Processing incremental hoodie file - " + filteredFile.getPath()); filteredFile = checkFileStatus(filteredFile); returns.add(filteredFile.getFileStatus()); } - LOG.info( - "Total paths to process after hoodie incremental filter " + filteredFiles.size()); + LOG.info("Total paths to process after hoodie incremental filter " + filteredFiles.size()); } else { // filter files on the latest commit found List filteredFiles = roView.getLatestDataFiles() @@ -171,13 +168,13 @@ public class HoodieInputFormat extends MapredParquetInputFormat String nonHoodieBasePath = null; for (FileStatus status : fileStatuses) { if (!status.getPath().getName().endsWith(".parquet")) { - //FIXME(vc): skip non parquet files for now. This wont be needed once log file name start with "." + //FIXME(vc): skip non parquet files for now. This wont be needed once log file name start + // with "." continue; } if ((metadata == null && nonHoodieBasePath == null) || (metadata == null && !status.getPath() - .toString() - .contains(nonHoodieBasePath)) || (metadata != null && !status.getPath().toString() - .contains(metadata.getBasePath()))) { + .toString().contains(nonHoodieBasePath)) || (metadata != null && !status.getPath() + .toString().contains(metadata.getBasePath()))) { try { metadata = getTableMetaClient(status.getPath().getFileSystem(conf), status.getPath().getParent()); @@ -185,8 +182,7 @@ public class HoodieInputFormat extends MapredParquetInputFormat } catch (InvalidDatasetException e) { LOG.info("Handling a non-hoodie path " + status.getPath()); metadata = null; - nonHoodieBasePath = - status.getPath().getParent().toString(); + nonHoodieBasePath = status.getPath().getParent().toString(); } if (!grouped.containsKey(metadata)) { grouped.put(metadata, new ArrayList<>()); @@ -209,17 +205,17 @@ public class HoodieInputFormat extends MapredParquetInputFormat public RecordReader getRecordReader(final InputSplit split, final JobConf job, final Reporter reporter) throws IOException { // TODO enable automatic predicate pushdown after fixing issues -// FileSplit fileSplit = (FileSplit) split; -// HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent()); -// String tableName = metadata.getTableName(); -// String mode = HoodieHiveUtil.readMode(job, tableName); + // FileSplit fileSplit = (FileSplit) split; + // HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent()); + // String tableName = metadata.getTableName(); + // String mode = HoodieHiveUtil.readMode(job, tableName); -// if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) { -// FilterPredicate predicate = constructHoodiePredicate(job, tableName, split); -// LOG.info("Setting parquet predicate push down as " + predicate); -// ParquetInputFormat.setFilterPredicate(job, predicate); + // if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) { + // FilterPredicate predicate = constructHoodiePredicate(job, tableName, split); + // LOG.info("Setting parquet predicate push down as " + predicate); + // ParquetInputFormat.setFilterPredicate(job, predicate); //clearOutExistingPredicate(job); -// } + // } return super.getRecordReader(split, job, reporter); } @@ -236,9 +232,8 @@ public class HoodieInputFormat extends MapredParquetInputFormat * `hoodie_commit_time` > 'start_commit_time' and ANDs with the existing predicate if one is * present already. */ - private FilterPredicate constructHoodiePredicate(JobConf job, - String tableName, - InputSplit split) throws IOException { + private FilterPredicate constructHoodiePredicate(JobConf job, String tableName, InputSplit split) + throws IOException { FilterPredicate commitTimePushdown = constructCommitTimePushdownPredicate(job, tableName); LOG.info("Commit time predicate - " + commitTimePushdown.toString()); FilterPredicate existingPushdown = constructHQLPushdownPredicate(job, split); @@ -262,21 +257,19 @@ public class HoodieInputFormat extends MapredParquetInputFormat || columnNamesString.isEmpty()) { return null; } else { - SearchArgument sarg = - SearchArgumentFactory.create(Utilities.deserializeExpression(serializedPushdown)); + SearchArgument sarg = SearchArgumentFactory + .create(Utilities.deserializeExpression(serializedPushdown)); final Path finalPath = ((FileSplit) split).getPath(); final ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(job, finalPath); final FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); - return ParquetFilterPredicateConverter - .toFilterPredicate(sarg, fileMetaData.getSchema()); + return ParquetFilterPredicateConverter.toFilterPredicate(sarg, fileMetaData.getSchema()); } } private FilterPredicate constructCommitTimePushdownPredicate(JobConf job, String tableName) throws IOException { String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName); - Operators.BinaryColumn sequenceColumn = - binaryColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD); + Operators.BinaryColumn sequenceColumn = binaryColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD); FilterPredicate p = gt(sequenceColumn, Binary.fromString(lastIncrementalTs)); LOG.info("Setting predicate in InputFormat " + p.toString()); return p; diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java index 2c01ed666..44672ec4b 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.hadoop; import com.uber.hoodie.common.model.HoodieDataFile; @@ -37,12 +38,12 @@ import org.apache.hadoop.fs.PathFilter; /** * Given a path is a part of - Hoodie dataset = accepts ONLY the latest version of each path - * Non-Hoodie dataset = then always accept - * + *

* We can set this filter, on a query engine's Hadoop Config and if it respects path filters, then * you should be able to query both hoodie and non-hoodie datasets as you would normally do. - * - * hadoopConf.setClass("mapreduce.input.pathFilter.class", com.uber.hoodie.hadoop.HoodieROTablePathFilter.class, - * org.apache.hadoop.fs.PathFilter.class) + *

+ * hadoopConf.setClass("mapreduce.input.pathFilter.class", com.uber.hoodie.hadoop + * .HoodieROTablePathFilter.class, org.apache.hadoop.fs.PathFilter.class) */ public class HoodieROTablePathFilter implements PathFilter, Serializable { @@ -104,8 +105,7 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable { if (hoodiePathCache.containsKey(folder.toString())) { if (LOG.isDebugEnabled()) { - LOG.debug(String.format("%s Hoodie path checked against cache, accept => %s \n", - path, + LOG.debug(String.format("%s Hoodie path checked against cache, accept => %s \n", path, hoodiePathCache.get(folder.toString()).contains(path))); } return hoodiePathCache.get(folder.toString()).contains(path); @@ -123,37 +123,33 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable { if (baseDir != null) { try { - HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(fs.getConf(), baseDir.toString()); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), + baseDir.toString()); HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline().getCommitTimeline() - .filterCompletedInstants(), + metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), fs.listStatus(folder)); - List latestFiles = fsView - .getLatestDataFiles() + List latestFiles = fsView.getLatestDataFiles() .collect(Collectors.toList()); // populate the cache if (!hoodiePathCache.containsKey(folder.toString())) { hoodiePathCache.put(folder.toString(), new HashSet<>()); } - LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() + - ", caching " + latestFiles.size() + " files under " + folder); + LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() + ", caching " + + latestFiles.size() + " files under " + folder); for (HoodieDataFile lfile : latestFiles) { hoodiePathCache.get(folder.toString()).add(new Path(lfile.getPath())); } // accept the path, if its among the latest files. if (LOG.isDebugEnabled()) { - LOG.debug(String.format("%s checked after cache population, accept => %s \n", - path, + LOG.debug(String.format("%s checked after cache population, accept => %s \n", path, hoodiePathCache.get(folder.toString()).contains(path))); } return hoodiePathCache.get(folder.toString()).contains(path); } catch (DatasetNotFoundException e) { // Non-hoodie path, accept it. if (LOG.isDebugEnabled()) { - LOG.debug(String.format("(1) Caching non-hoodie path under %s \n", - folder.toString())); + LOG.debug(String.format("(1) Caching non-hoodie path under %s \n", folder.toString())); } nonHoodiePathCache.add(folder.toString()); return true; diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java index 939f58297..a6f12b8f9 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java @@ -79,8 +79,7 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf // TODO(vc): Should we handle also non-hoodie splits here? Map metaClientMap = new HashMap<>(); Map partitionsToMetaClient = partitionsToParquetSplits.keySet() - .stream() - .collect(Collectors.toMap(Function.identity(), p -> { + .stream().collect(Collectors.toMap(Function.identity(), p -> { // find if we have a metaclient already for this partition. Optional matchingBasePath = metaClientMap.keySet().stream() .filter(basePath -> p.toString().startsWith(basePath)).findFirst(); @@ -97,7 +96,8 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf } })); - // for all unique split parents, obtain all delta files based on delta commit timeline, grouped on file id + // for all unique split parents, obtain all delta files based on delta commit timeline, + // grouped on file id List rtSplits = new ArrayList<>(); partitionsToParquetSplits.keySet().stream().forEach(partitionPath -> { // for each partition path obtain the data & log file groupings, then map back to inputsplits @@ -119,14 +119,13 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf dataFileSplits.forEach(split -> { try { List logFilePaths = fileSlice.getLogFiles() - .map(logFile -> logFile.getPath().toString()) - .collect(Collectors.toList()); - // Get the maxCommit from the last delta or compaction or commit - when bootstrapped from COW table - String maxCommitTime = metaClient.getActiveTimeline() - .getTimelineOfActions( - Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, - HoodieTimeline.DELTA_COMMIT_ACTION)) - .filterCompletedInstants().lastInstant().get().getTimestamp(); + .map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()); + // Get the maxCommit from the last delta or compaction or commit - when + // bootstrapped from COW table + String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions( + Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, + HoodieTimeline.DELTA_COMMIT_ACTION)).filterCompletedInstants().lastInstant() + .get().getTimestamp(); rtSplits.add( new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths, maxCommitTime)); @@ -147,7 +146,8 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf @Override public FileStatus[] listStatus(JobConf job) throws IOException { - // Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit timeline. + // Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit + // timeline. return super.listStatus(job); } @@ -170,12 +170,11 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf if (!readColNames.contains(fieldName)) { // If not already in the list - then add it - conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, - readColNamesPrefix + fieldName); + conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, readColNamesPrefix + fieldName); conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, readColIdsPrefix + fieldIndex); if (LOG.isDebugEnabled()) { - LOG.debug(String.format("Adding extra column " + fieldName - + ", to enable log merging cols (%s) ids (%s) ", + LOG.debug(String.format( + "Adding extra column " + fieldName + ", to enable log merging cols (%s) ids (%s) ", conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR))); } @@ -189,15 +188,14 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf HOODIE_RECORD_KEY_COL_POS); configuration = addProjectionField(configuration, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HOODIE_COMMIT_TIME_COL_POS); - configuration = addProjectionField(configuration, - HoodieRecord.PARTITION_PATH_METADATA_FIELD, HOODIE_PARTITION_PATH_COL_POS); + configuration = addProjectionField(configuration, HoodieRecord.PARTITION_PATH_METADATA_FIELD, + HOODIE_PARTITION_PATH_COL_POS); return configuration; } @Override public RecordReader getRecordReader(final InputSplit split, - final JobConf job, - final Reporter reporter) throws IOException { + final JobConf job, final Reporter reporter) throws IOException { LOG.info("Creating record reader with readCols :" + job .get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); // sanity check diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java index 81cae359b..357023b4b 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -73,8 +73,10 @@ public class HoodieRealtimeRecordReader implements RecordReader deltaRecordMap; private final MessageType baseFileSchema; - public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, - JobConf job, + public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job, RecordReader realReader) { this.split = split; this.jobConf = job; @@ -106,11 +107,9 @@ public class HoodieRealtimeRecordReader implements RecordReader the commit we are trying to read (if using readCommit() API) + // but can return records for completed commits > the commit we are trying to read (if using + // readCommit() API) for (HoodieRecord hoodieRecord : compactedLogRecordScanner) { - GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(readerSchema) - .get(); + GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(readerSchema).get(); String key = hoodieRecord.getRecordKey(); - // we assume, a later safe record in the log, is newer than what we have in the map & replace it. + // we assume, a later safe record in the log, is newer than what we have in the map & + // replace it. // TODO : handle deletes here ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, writerSchema); deltaRecordMap.put(key, aWritable); @@ -180,14 +179,13 @@ public class HoodieRealtimeRecordReader implements RecordReader partitioningFields = Arrays.stream(partitioningFieldsCsv.split(",")) .collect(Collectors.toSet()); List fieldNames = Arrays.stream(fieldNameCsv.split(",")) - .filter(fn -> !partitioningFields.contains(fn)).collect( - Collectors.toList()); + .filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList()); // Hive does not provide ids for partitioning fields, so check for lengths excluding that. if (fieldNames.size() != fieldOrders.length) { - throw new HoodieException(String.format( - "Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d", - fieldNames.size(), fieldOrders.length)); + throw new HoodieException(String + .format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d", + fieldNames.size(), fieldOrders.length)); } TreeMap orderedFieldMap = new TreeMap<>(); for (int ox = 0; ox < fieldOrders.length; ox++) { @@ -287,26 +285,28 @@ public class HoodieRealtimeRecordReader implements RecordReader - recordReader = inputFormat.getRecordReader(split, jobConf, null); + RecordReader recordReader = inputFormat + .getRecordReader(split, jobConf, null); Void key = recordReader.createKey(); ArrayWritable writable = recordReader.createValue(); while (recordReader.next(key, writable)) { - // writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno] + // writable returns an array with [field1, field2, _hoodie_commit_time, + // _hoodie_commit_seqno] // Take the commit time and compare with the one we are interested in if (commit.equals((writable.get()[2]).toString())) { actualCount++; diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java index c8305e3a3..4088afdd0 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java @@ -42,16 +42,15 @@ public class InputFormatTestUtil { HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); for (int i = 0; i < numberOfFiles; i++) { - File dataFile = - new File(partitionPath, FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i)); + File dataFile = new File(partitionPath, + FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i)); dataFile.createNewFile(); } return partitionPath; } public static void simulateUpdates(File directory, final String originalCommit, - int numberOfFilesUpdated, - String newCommit, boolean randomize) throws IOException { + int numberOfFilesUpdated, String newCommit, boolean randomize) throws IOException { List dataFiles = Arrays.asList(directory.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { @@ -62,8 +61,8 @@ public class InputFormatTestUtil { if (randomize) { Collections.shuffle(dataFiles); } - List toUpdateList = - dataFiles.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size())); + List toUpdateList = dataFiles + .subList(0, Math.min(numberOfFilesUpdated, dataFiles.size())); for (File file : toUpdateList) { String fileId = FSUtils.getFileId(file.getName()); File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, 1, fileId)); @@ -78,8 +77,8 @@ public class InputFormatTestUtil { public static void setupIncremental(JobConf jobConf, String startCommit, int numberOfCommitsToPull) { - String modePropertyName = String.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, - HoodieTestUtils.RAW_TRIPS_TEST_NAME); + String modePropertyName = String + .format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME); jobConf.set(modePropertyName, HoodieHiveUtil.INCREMENTAL_SCAN_MODE); String startCommitTimestampName = String @@ -96,19 +95,16 @@ public class InputFormatTestUtil { } public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema, - int numberOfFiles, int numberOfRecords, - String commitNumber) throws IOException { + int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException { basePath.create(); HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); AvroParquetWriter parquetWriter; for (int i = 0; i < numberOfFiles; i++) { String fileId = FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i); - File dataFile = - new File(partitionPath, fileId); + File dataFile = new File(partitionPath, fileId); // dataFile.createNewFile(); - parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), - schema); + parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema); try { for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber, fileId)) { @@ -132,8 +128,7 @@ public class InputFormatTestUtil { } public static void simulateParquetUpdates(File directory, Schema schema, String originalCommit, - int totalNumberOfRecords, int numberOfRecordsToUpdate, - String newCommit) throws IOException { + int totalNumberOfRecords, int numberOfRecordsToUpdate, String newCommit) throws IOException { File fileToUpdate = directory.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { @@ -145,8 +140,8 @@ public class InputFormatTestUtil { AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema); try { - for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, - originalCommit, fileId)) { + for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit, + fileId)) { if (numberOfRecordsToUpdate > 0) { // update this record record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, newCommit); diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java index e64f918d4..c81f9fccb 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/TestHoodieROTablePathFilter.java @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.uber.hoodie.hadoop; import static org.junit.Assert.assertFalse; diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index 653e122d3..8e43aa25e 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -18,7 +18,6 @@ package com.uber.hoodie.hadoop.realtime; - import static org.junit.Assert.assertTrue; import com.google.common.collect.Maps; @@ -85,9 +84,7 @@ public class HoodieRealtimeRecordReaderTest { HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() .onParentPath(new Path(partitionDir.getPath())) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId) - .overBaseCommit(baseCommit) - .withFs(fs) - .build(); + .overBaseCommit(baseCommit).withFs(fs).build(); List records = new ArrayList<>(); for (int i = 0; i < numberOfRecords; i++) { records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0")); @@ -106,8 +103,8 @@ public class HoodieRealtimeRecordReaderTest { public void testReader() throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema()); - HoodieTestUtils - .initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), + HoodieTableType.MERGE_ON_READ); String commitTime = "100"; File partitionDir = InputFormatTestUtil .prepareParquetDataset(basePath, schema, 1, 100, commitTime); @@ -125,15 +122,15 @@ public class HoodieRealtimeRecordReaderTest { //create a split with baseFile (parquet file written earlier) and new log file(s) String logFilePath = writer.getLogFile().getPath().toString(); - HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(new FileSplit(new Path(partitionDir - + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, jobConf), basePath.getRoot().getPath(), - Arrays.asList(logFilePath), newCommitTime); + HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( + new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, + jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime); //create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = - new MapredParquetInputFormat(). - getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), - (String[]) null), jobConf, null); + new MapredParquetInputFormat().getRecordReader( + new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), + jobConf, null); JobConf jobConf = new JobConf(); List fields = schema.getFields(); String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); @@ -164,8 +161,8 @@ public class HoodieRealtimeRecordReaderTest { public void testReaderWithNestedAndComplexSchema() throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema()); - HoodieTestUtils - .initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), + HoodieTableType.MERGE_ON_READ); String commitTime = "100"; int numberOfRecords = 100; int numberOfLogRecords = numberOfRecords / 2; @@ -185,15 +182,15 @@ public class HoodieRealtimeRecordReaderTest { //create a split with baseFile (parquet file written earlier) and new log file(s) String logFilePath = writer.getLogFile().getPath().toString(); - HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(new FileSplit(new Path(partitionDir - + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, jobConf), basePath.getRoot().getPath(), - Arrays.asList(logFilePath), newCommitTime); + HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( + new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, + jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime); //create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = - new MapredParquetInputFormat(). - getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), - (String[]) null), jobConf, null); + new MapredParquetInputFormat().getRecordReader( + new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), + jobConf, null); JobConf jobConf = new JobConf(); List fields = schema.getFields(); diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java index 5379580ef..dd9e70149 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncConfig.java @@ -48,15 +48,23 @@ public class HiveSyncConfig implements Serializable { "--base-path"}, description = "Basepath of hoodie dataset to sync", required = true) public String basePath; - @Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by", required = true) + @Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by", + required = true) public List partitionFields = new ArrayList<>(); - @Parameter(names = "-partition-value-extractor", description = "Class which implements PartitionValueExtractor to extract the partition values from HDFS path") + @Parameter(names = "-partition-value-extractor", description = "Class which implements " + + "PartitionValueExtractor " + + "to extract the partition " + + "values from HDFS path") public String partitionValueExtractorClass = SlashEncodedDayPartitionValueExtractor.class .getName(); @Parameter(names = { - "--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter") + "--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this" + + " exists to support " + + "backward compatibility. If" + + " you use hoodie 0.3.x, do " + + "not set this parameter") public Boolean assumeDatePartitioning = false; @Parameter(names = {"--help", "-h"}, help = true) diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java index 472001ede..164aa7a6c 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java @@ -45,16 +45,16 @@ import parquet.schema.MessageType; * Tool to sync a hoodie HDFS dataset with a hive metastore table. Either use it as a api * HiveSyncTool.syncHoodieTable(HiveSyncConfig) or as a command line java -cp hoodie-hive.jar * HiveSyncTool [args] - * + *

* This utility will get the schema from the latest commit and will sync hive table schema Also this * will sync the partitions incrementally (all the partitions modified since the last commit) */ @SuppressWarnings("WeakerAccess") public class HiveSyncTool { - private static Logger LOG = LoggerFactory.getLogger(HiveSyncTool.class); + private static final Logger LOG = LoggerFactory.getLogger(HiveSyncTool.class); private final HoodieHiveClient hoodieHiveClient; - public final static String SUFFIX_REALTIME_TABLE = "_rt"; + public static final String SUFFIX_REALTIME_TABLE = "_rt"; private final HiveSyncConfig cfg; public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { @@ -85,9 +85,8 @@ public class HiveSyncTool { } private void syncHoodieTable(boolean isRealTime) { - LOG.info("Trying to sync hoodie table " + cfg.tableName + " with base path " + hoodieHiveClient - .getBasePath() + " of type " + hoodieHiveClient - .getTableType()); + LOG.info("Trying to sync hoodie table " + cfg.tableName + " with base path " + + hoodieHiveClient.getBasePath() + " of type " + hoodieHiveClient.getTableType()); // Check if the necessary table exists boolean tableExists = hoodieHiveClient.doesTableExist(); @@ -118,27 +117,29 @@ public class HiveSyncTool { * If not, evolves the table schema. * * @param tableExists - does table exist - * @param schema - extracted schema + * @param schema - extracted schema */ private void syncSchema(boolean tableExists, boolean isRealTime, MessageType schema) { // Check and sync schema if (!tableExists) { LOG.info("Table " + cfg.tableName + " is not found. Creating it"); if (!isRealTime) { - // TODO - RO Table for MOR only after major compaction (UnboundedCompaction is default for now) + // TODO - RO Table for MOR only after major compaction (UnboundedCompaction is default + // for now) hoodieHiveClient.createTable(schema, HoodieInputFormat.class.getName(), MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName()); } else { // Custom serde will not work with ALTER TABLE REPLACE COLUMNS - // https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java#L3488 + // https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive + // /ql/exec/DDLTask.java#L3488 hoodieHiveClient.createTable(schema, HoodieRealtimeInputFormat.class.getName(), MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName()); } } else { // Check if the dataset schema has evolved Map tableSchema = hoodieHiveClient.getTableSchema(); - SchemaDifference schemaDiff = SchemaUtil - .getSchemaDifference(schema, tableSchema, cfg.partitionFields); + SchemaDifference schemaDiff = SchemaUtil.getSchemaDifference(schema, tableSchema, + cfg.partitionFields); if (!schemaDiff.isEmpty()) { LOG.info("Schema difference found for " + cfg.tableName); hoodieHiveClient.updateTableDefinition(schema); @@ -156,8 +157,8 @@ public class HiveSyncTool { private void syncPartitions(List writtenPartitionsSince) { try { List hivePartitions = hoodieHiveClient.scanTablePartitions(); - List partitionEvents = hoodieHiveClient - .getPartitionEvents(hivePartitions, writtenPartitionsSince); + List partitionEvents = hoodieHiveClient.getPartitionEvents(hivePartitions, + writtenPartitionsSince); List newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD); LOG.info("New Partitions " + newPartitions); hoodieHiveClient.addPartitionsToTable(newPartitions); @@ -165,15 +166,13 @@ public class HiveSyncTool { LOG.info("Changed Partitions " + updatePartitions); hoodieHiveClient.updatePartitionsToTable(updatePartitions); } catch (Exception e) { - throw new HoodieHiveSyncException("Failed to sync partitions for table " + cfg.tableName, - e); + throw new HoodieHiveSyncException("Failed to sync partitions for table " + cfg.tableName, e); } } private List filterPartitions(List events, PartitionEventType eventType) { - return events.stream() - .filter(s -> s.eventType == eventType).map(s -> s.storagePartition).collect( - Collectors.toList()); + return events.stream().filter(s -> s.eventType == eventType).map(s -> s.storagePartition) + .collect(Collectors.toList()); } public static void main(String[] args) throws Exception { diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index 1ac16ad43..c3d202b5b 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -15,6 +15,7 @@ * * */ + package com.uber.hoodie.hive; import com.google.common.base.Preconditions; @@ -25,15 +26,22 @@ import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieTableType; 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.Reader; -import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; -import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.InvalidDatasetException; import com.uber.hoodie.hive.util.SchemaUtil; +import java.io.IOException; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; import org.apache.commons.dbcp.BasicDataSource; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -51,18 +59,6 @@ import parquet.hadoop.ParquetFileReader; import parquet.hadoop.metadata.ParquetMetadata; import parquet.schema.MessageType; -import java.io.IOException; -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; - @SuppressWarnings("ConstantConditions") public class HoodieHiveClient { @@ -103,16 +99,15 @@ public class HoodieHiveClient { } try { - this.partitionValueExtractor = (PartitionValueExtractor) Class - .forName(cfg.partitionValueExtractorClass).newInstance(); + this.partitionValueExtractor = (PartitionValueExtractor) Class.forName( + cfg.partitionValueExtractorClass).newInstance(); } catch (Exception e) { throw new HoodieHiveSyncException( "Failed to initialize PartitionValueExtractor class " + cfg.partitionValueExtractorClass, e); } - activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants(); + activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); } public HoodieTimeline getActiveTimeline() { @@ -207,9 +202,8 @@ public class HoodieHiveClient { for (Partition tablePartition : tablePartitions) { List hivePartitionValues = tablePartition.getValues(); Collections.sort(hivePartitionValues); - String fullTablePartitionPath = Path - .getPathWithoutSchemeAndAuthority(new Path(tablePartition.getSd().getLocation())).toUri() - .getPath(); + String fullTablePartitionPath = Path.getPathWithoutSchemeAndAuthority( + new Path(tablePartition.getSd().getLocation())).toUri().getPath(); paths.put(String.join(", ", hivePartitionValues), fullTablePartitionPath); } @@ -235,8 +229,7 @@ public class HoodieHiveClient { * Scan table partitions */ List scanTablePartitions() throws TException { - return client - .listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1); + return client.listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1); } void updateTableDefinition(MessageType newSchema) { @@ -245,9 +238,10 @@ public class HoodieHiveClient { // Cascade clause should not be present for non-partitioned tables String cascadeClause = syncConfig.partitionFields.size() > 0 ? " cascade" : ""; StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append("`") - .append(syncConfig.databaseName).append(".").append(syncConfig.tableName).append("`") - .append(" REPLACE COLUMNS(") - .append(newSchemaStr).append(" )").append(cascadeClause); + .append(syncConfig.databaseName).append(".") + .append(syncConfig.tableName).append("`") + .append(" REPLACE COLUMNS(").append(newSchemaStr).append(" )") + .append(cascadeClause); LOG.info("Creating table with " + sqlBuilder); updateHiveSQL(sqlBuilder.toString()); } catch (IOException e) { @@ -255,8 +249,8 @@ public class HoodieHiveClient { } } - void createTable(MessageType storageSchema, - String inputFormatClass, String outputFormatClass, String serdeClass) { + void createTable(MessageType storageSchema, String inputFormatClass, String outputFormatClass, + String serdeClass) { try { String createSQLQuery = SchemaUtil .generateCreateDDL(storageSchema, syncConfig, inputFormatClass, @@ -289,8 +283,8 @@ public class HoodieHiveClient { } return schema; } catch (SQLException e) { - throw new HoodieHiveSyncException( - "Failed to get table schema for " + syncConfig.tableName, e); + throw new HoodieHiveSyncException("Failed to get table schema for " + syncConfig.tableName, + e); } finally { closeQuietly(result, null); } @@ -308,49 +302,47 @@ public class HoodieHiveClient { try { switch (tableType) { case COPY_ON_WRITE: - // If this is COW, get the last commit and read the schema from a file written in the last commit - HoodieInstant lastCommit = activeTimeline.lastInstant() - .orElseThrow(() -> new InvalidDatasetException(syncConfig.basePath)); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(activeTimeline.getInstantDetails(lastCommit).get()); + // If this is COW, get the last commit and read the schema from a file written in the + // last commit + HoodieInstant lastCommit = activeTimeline.lastInstant().orElseThrow( + () -> new InvalidDatasetException(syncConfig.basePath)); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + activeTimeline.getInstantDetails(lastCommit).get()); String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() - .stream().findAny() - .orElseThrow(() -> new IllegalArgumentException( + .stream().findAny().orElseThrow(() -> new IllegalArgumentException( "Could not find any data file written for commit " + lastCommit + ", could not get schema for dataset " + metaClient.getBasePath())); return readSchemaFromDataFile(new Path(filePath)); case MERGE_ON_READ: - // If this is MOR, depending on whether the latest commit is a delta commit or compaction commit + // If this is MOR, depending on whether the latest commit is a delta commit or + // compaction commit // Get a datafile written and get the schema from that file Optional lastCompactionCommit = metaClient.getActiveTimeline() - .getCommitTimeline().filterCompletedInstants().lastInstant(); + .getCommitTimeline() + .filterCompletedInstants() + .lastInstant(); LOG.info("Found the last compaction commit as " + lastCompactionCommit); Optional lastDeltaCommit; if (lastCompactionCommit.isPresent()) { - lastDeltaCommit = metaClient.getActiveTimeline() - .getDeltaCommitTimeline() + lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline() .filterCompletedInstants() - .findInstantsAfter(lastCompactionCommit.get().getTimestamp(), Integer.MAX_VALUE) - .lastInstant(); + .findInstantsAfter(lastCompactionCommit.get().getTimestamp(), + Integer.MAX_VALUE).lastInstant(); } else { - lastDeltaCommit = metaClient.getActiveTimeline() - .getDeltaCommitTimeline() - .filterCompletedInstants() - .lastInstant(); + lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline() + .filterCompletedInstants().lastInstant(); } - LOG.info("Found the last delta commit " - + lastDeltaCommit); + LOG.info("Found the last delta commit " + lastDeltaCommit); if (lastDeltaCommit.isPresent()) { HoodieInstant lastDeltaInstant = lastDeltaCommit.get(); // read from the log file wrote - commitMetadata = HoodieCommitMetadata - .fromBytes(activeTimeline.getInstantDetails(lastDeltaInstant).get()); + commitMetadata = HoodieCommitMetadata.fromBytes( + activeTimeline.getInstantDetails(lastDeltaInstant).get()); filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() - .stream().filter(s -> s.contains( - HoodieLogFile.DELTA_EXTENSION)).findAny() - .orElseThrow(() -> new IllegalArgumentException( + .stream().filter(s -> s.contains(HoodieLogFile.DELTA_EXTENSION)) + .findAny().orElseThrow(() -> new IllegalArgumentException( "Could not find any data file written for commit " + lastDeltaInstant + ", could not get schema for dataset " + metaClient.getBasePath())); return readSchemaFromLogFile(lastCompactionCommit, new Path(filePath)); @@ -362,8 +354,8 @@ public class HoodieHiveClient { throw new InvalidDatasetException(syncConfig.basePath); } } catch (IOException e) { - throw new HoodieHiveSyncException( - "Failed to get dataset schema for " + syncConfig.tableName, e); + throw new HoodieHiveSyncException("Failed to get dataset schema for " + syncConfig.tableName, + e); } } @@ -379,11 +371,10 @@ public class HoodieHiveClient { + syncConfig.basePath)); // Read from the compacted file wrote - HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata - .fromBytes(activeTimeline.getInstantDetails(lastCompactionCommit).get()); + HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata.fromBytes( + activeTimeline.getInstantDetails(lastCompactionCommit).get()); String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values() - .stream().findAny() - .orElseThrow(() -> new IllegalArgumentException( + .stream().findAny().orElseThrow(() -> new IllegalArgumentException( "Could not find any data file written for compaction " + lastCompactionCommit + ", could not get schema for dataset " + metaClient.getBasePath())); return readSchemaFromDataFile(new Path(filePath)); @@ -411,12 +402,10 @@ public class HoodieHiveClient { LOG.info("Reading schema from " + parquetFilePath); if (!fs.exists(parquetFilePath)) { throw new IllegalArgumentException( - "Failed to read schema from data file " + parquetFilePath - + ". File does not exist."); + "Failed to read schema from data file " + parquetFilePath + ". File does not exist."); } - ParquetMetadata fileFooter = - ParquetFileReader - .readFooter(fs.getConf(), parquetFilePath, ParquetMetadataConverter.NO_FILTER); + ParquetMetadata fileFooter = ParquetFileReader.readFooter(fs.getConf(), parquetFilePath, + ParquetMetadataConverter.NO_FILTER); return fileFooter.getFileMetaData().getSchema(); } @@ -427,8 +416,8 @@ public class HoodieHiveClient { try { return client.tableExists(syncConfig.databaseName, syncConfig.tableName); } catch (TException e) { - throw new HoodieHiveSyncException( - "Failed to check if table exists " + syncConfig.tableName, e); + throw new HoodieHiveSyncException("Failed to check if table exists " + syncConfig.tableName, + e); } } @@ -511,8 +500,8 @@ public class HoodieHiveClient { // Get the last commit time from the TBLproperties try { Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName); - return Optional - .ofNullable(database.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null)); + return Optional.ofNullable( + database.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null)); } catch (Exception e) { throw new HoodieHiveSyncException( "Failed to get the last commit time synced from the database", e); @@ -537,8 +526,8 @@ public class HoodieHiveClient { if (!lastCommitTimeSynced.isPresent()) { LOG.info("Last commit time synced is not known, listing all partitions"); try { - return FSUtils - .getAllPartitionPaths(fs, syncConfig.basePath, syncConfig.assumeDatePartitioning); + return FSUtils.getAllPartitionPaths(fs, syncConfig.basePath, + syncConfig.assumeDatePartitioning); } catch (IOException e) { throw new HoodieIOException("Failed to list all partitions in " + syncConfig.basePath, e); } @@ -546,8 +535,8 @@ public class HoodieHiveClient { LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then"); - HoodieTimeline timelineToSync = activeTimeline - .findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE); + HoodieTimeline timelineToSync = activeTimeline.findInstantsAfter(lastCommitTimeSynced.get(), + Integer.MAX_VALUE); return timelineToSync.getInstants().map(s -> { try { return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(s).get()); @@ -579,13 +568,15 @@ public class HoodieHiveClient { */ static class PartitionEvent { - public enum PartitionEventType {ADD, UPDATE} + public enum PartitionEventType { + ADD, + UPDATE + } PartitionEventType eventType; String storagePartition; - PartitionEvent( - PartitionEventType eventType, String storagePartition) { + PartitionEvent(PartitionEventType eventType, String storagePartition) { this.eventType = eventType; this.storagePartition = storagePartition; } diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java index 794c262e3..73776c461 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/PartitionValueExtractor.java @@ -24,8 +24,9 @@ import java.util.List; * HDFS Path contain hive partition values for the keys it is partitioned on. This mapping is not * straight forward and requires a pluggable implementation to extract the partition value from HDFS * path. - * - * e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd] + *

+ * e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path + * /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd] */ public interface PartitionValueExtractor { diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java index 956bbb8b2..893b61e4c 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/SlashEncodedDayPartitionValueExtractor.java @@ -28,7 +28,7 @@ import org.joda.time.format.DateTimeFormatter; * HDFS Path contain hive partition values for the keys it is partitioned on. This mapping is not * straight forward and requires a pluggable implementation to extract the partition value from HDFS * path. - * + *

* This implementation extracts datestr=yyyy-mm-dd from path of type /yyyy/mm/dd */ public class SlashEncodedDayPartitionValueExtractor implements PartitionValueExtractor { diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/ColumnNameXLator.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/ColumnNameXLator.java index a06494fe2..cc7039a38 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/ColumnNameXLator.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/ColumnNameXLator.java @@ -26,9 +26,9 @@ public class ColumnNameXLator { public static String translateNestedColumn(String colName) { Map.Entry entry; - for (Iterator i$ = xformMap.entrySet().iterator(); i$.hasNext(); + for (Iterator ic = xformMap.entrySet().iterator(); ic.hasNext(); colName = colName.replaceAll((String) entry.getKey(), (String) entry.getValue())) { - entry = (Map.Entry) i$.next(); + entry = (Map.Entry) ic.next(); } return colName; diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java index 9eda294d9..098c01368 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/util/SchemaUtil.java @@ -23,14 +23,12 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat; import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader; import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock; -import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.hive.HiveSyncConfig; import com.uber.hoodie.hive.HoodieHiveSyncException; import com.uber.hoodie.hive.SchemaDifference; import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -48,7 +46,7 @@ import parquet.schema.Type; */ public class SchemaUtil { - private static Logger LOG = LoggerFactory.getLogger(SchemaUtil.class); + private static final Logger LOG = LoggerFactory.getLogger(SchemaUtil.class); /** * Get the schema difference between the storage schema and hive table schema @@ -59,19 +57,19 @@ public class SchemaUtil { try { newTableSchema = convertParquetSchemaToHiveSchema(storageSchema); } catch (IOException e) { - throw new HoodieHiveSyncException("Failed to convert parquet schema to hive schema", - e); + throw new HoodieHiveSyncException("Failed to convert parquet schema to hive schema", e); } LOG.info("Getting schema difference for " + tableSchema + "\r\n\r\n" + newTableSchema); - SchemaDifference.Builder schemaDiffBuilder = - SchemaDifference.newBuilder(storageSchema, tableSchema); + SchemaDifference.Builder schemaDiffBuilder = SchemaDifference + .newBuilder(storageSchema, tableSchema); Set tableColumns = Sets.newHashSet(); for (Map.Entry field : tableSchema.entrySet()) { String fieldName = field.getKey().toLowerCase(); String tickSurroundedFieldName = tickSurround(fieldName); if (!isFieldExistsInSchema(newTableSchema, tickSurroundedFieldName) && !partitionKeys - .contains(fieldName)) { + .contains( + fieldName)) { schemaDiffBuilder.deleteTableColumn(fieldName); } else { // check type @@ -82,8 +80,8 @@ public class SchemaUtil { continue; } // We will log this and continue. Hive schema is a superset of all parquet schemas - LOG.warn("Ignoring table column " + fieldName - + " as its not present in the parquet schema"); + LOG.warn( + "Ignoring table column " + fieldName + " as its not present in the parquet schema"); continue; } tableColumnType = tableColumnType.replaceAll("\\s+", ""); @@ -93,11 +91,12 @@ public class SchemaUtil { expectedType = expectedType.replaceAll("`", ""); if (!tableColumnType.equalsIgnoreCase(expectedType)) { - // check for incremental datasets, the schema type change is allowed as per evolution rules + // check for incremental datasets, the schema type change is allowed as per evolution + // rules if (!isSchemaTypeUpdateAllowed(tableColumnType, expectedType)) { throw new HoodieHiveSyncException( - "Could not convert field Type from " + tableColumnType + " to " - + expectedType + " for field " + fieldName); + "Could not convert field Type from " + tableColumnType + " to " + expectedType + + " for field " + fieldName); } schemaDiffBuilder.updateTableColumn(fieldName, getExpectedType(newTableSchema, tickSurroundedFieldName)); @@ -169,21 +168,19 @@ public class SchemaUtil { private static String convertField(final Type parquetType) { StringBuilder field = new StringBuilder(); if (parquetType.isPrimitive()) { - final PrimitiveType.PrimitiveTypeName parquetPrimitiveTypeName = - parquetType.asPrimitiveType().getPrimitiveTypeName(); + final PrimitiveType.PrimitiveTypeName parquetPrimitiveTypeName = parquetType.asPrimitiveType() + .getPrimitiveTypeName(); final OriginalType originalType = parquetType.getOriginalType(); if (originalType == OriginalType.DECIMAL) { - final DecimalMetadata decimalMetadata = - parquetType.asPrimitiveType().getDecimalMetadata(); - return field.append("DECIMAL(").append(decimalMetadata.getPrecision()). - append(" , ").append(decimalMetadata.getScale()).append(")").toString(); + final DecimalMetadata decimalMetadata = parquetType.asPrimitiveType().getDecimalMetadata(); + return field.append("DECIMAL(").append(decimalMetadata.getPrecision()).append(" , ") + .append(decimalMetadata.getScale()).append(")").toString(); } // TODO - fix the method naming here return parquetPrimitiveTypeName .convert(new PrimitiveType.PrimitiveTypeNameConverter() { @Override - public String convertBOOLEAN( - PrimitiveType.PrimitiveTypeName primitiveTypeName) { + public String convertBOOLEAN(PrimitiveType.PrimitiveTypeName primitiveTypeName) { return "boolean"; } @@ -220,8 +217,7 @@ public class SchemaUtil { @Override public String convertBINARY(PrimitiveType.PrimitiveTypeName primitiveTypeName) { - if (originalType == OriginalType.UTF8 - || originalType == OriginalType.ENUM) { + if (originalType == OriginalType.UTF8 || originalType == OriginalType.ENUM) { return "string"; } else { return "binary"; @@ -235,33 +231,28 @@ public class SchemaUtil { switch (originalType) { case LIST: if (parquetGroupType.getFieldCount() != 1) { - throw new UnsupportedOperationException( - "Invalid list type " + parquetGroupType); + throw new UnsupportedOperationException("Invalid list type " + parquetGroupType); } Type elementType = parquetGroupType.getType(0); if (!elementType.isRepetition(Type.Repetition.REPEATED)) { - throw new UnsupportedOperationException( - "Invalid list type " + parquetGroupType); + throw new UnsupportedOperationException("Invalid list type " + parquetGroupType); } return createHiveArray(elementType, parquetGroupType.getName()); case MAP: if (parquetGroupType.getFieldCount() != 1 || parquetGroupType.getType(0) .isPrimitive()) { - throw new UnsupportedOperationException( - "Invalid map type " + parquetGroupType); + throw new UnsupportedOperationException("Invalid map type " + parquetGroupType); } GroupType mapKeyValType = parquetGroupType.getType(0).asGroupType(); - if (!mapKeyValType.isRepetition(Type.Repetition.REPEATED) || - !mapKeyValType.getOriginalType().equals(OriginalType.MAP_KEY_VALUE) || - mapKeyValType.getFieldCount() != 2) { - throw new UnsupportedOperationException( - "Invalid map type " + parquetGroupType); + if (!mapKeyValType.isRepetition(Type.Repetition.REPEATED) + || !mapKeyValType.getOriginalType().equals(OriginalType.MAP_KEY_VALUE) + || mapKeyValType.getFieldCount() != 2) { + throw new UnsupportedOperationException("Invalid map type " + parquetGroupType); } Type keyType = mapKeyValType.getType(0); - if (!keyType.isPrimitive() || - !keyType.asPrimitiveType().getPrimitiveTypeName() - .equals(PrimitiveType.PrimitiveTypeName.BINARY) || - !keyType.getOriginalType().equals(OriginalType.UTF8)) { + if (!keyType.isPrimitive() || !keyType.asPrimitiveType().getPrimitiveTypeName() + .equals(PrimitiveType.PrimitiveTypeName.BINARY) + || !keyType.getOriginalType().equals(OriginalType.UTF8)) { throw new UnsupportedOperationException( "Map key type must be binary (UTF8): " + keyType); } @@ -277,8 +268,7 @@ public class SchemaUtil { // MAP. Hence, PARQUET-113 // dropped the requirement for having MAP_KEY_VALUE. default: - throw new UnsupportedOperationException( - "Cannot convert Parquet type " + parquetType); + throw new UnsupportedOperationException("Cannot convert Parquet type " + parquetType); } } else { // if no original type then it's a record @@ -306,7 +296,8 @@ public class SchemaUtil { // ", " struct.append(">"); String finalStr = struct.toString(); - // Struct cannot have - in them. userstore_udr_entities has uuid in struct. This breaks the schema. + // Struct cannot have - in them. userstore_udr_entities has uuid in struct. This breaks the + // schema. // HDrone sync should not fail because of this. finalStr = finalStr.replaceAll("-", "_"); return finalStr; @@ -362,8 +353,8 @@ public class SchemaUtil { } public static boolean isSchemaTypeUpdateAllowed(String prevType, String newType) { - if (prevType == null || prevType.trim().isEmpty() || - newType == null || newType.trim().isEmpty()) { + if (prevType == null || prevType.trim().isEmpty() || newType == null || newType.trim() + .isEmpty()) { return false; } prevType = prevType.toLowerCase(); @@ -392,9 +383,8 @@ public class SchemaUtil { return columns.toString(); } - public static String generateCreateDDL(MessageType storageSchema, - HiveSyncConfig config, String inputFormatClass, - String outputFormatClass, String serdeClass) throws IOException { + public static String generateCreateDDL(MessageType storageSchema, HiveSyncConfig config, + String inputFormatClass, String outputFormatClass, String serdeClass) throws IOException { Map hiveSchema = convertParquetSchemaToHiveSchema(storageSchema); String columns = generateSchemaString(storageSchema); @@ -422,7 +412,8 @@ public class SchemaUtil { return hiveSchema.get(partitionKey); } // Default the unknown partition fields to be String - // TODO - all partition fields should be part of the schema. datestr is treated as special. Dont do that + // TODO - all partition fields should be part of the schema. datestr is treated as special. + // Dont do that return "String"; } @@ -430,8 +421,7 @@ public class SchemaUtil { * Read the schema from the log file on path */ @SuppressWarnings("OptionalUsedAsFieldOrParameterType") - public static MessageType readSchemaFromLogFile(FileSystem fs, - Path path) throws IOException { + public static MessageType readSchemaFromLogFile(FileSystem fs, Path path) throws IOException { Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null); HoodieAvroDataBlock lastBlock = null; while (reader.hasNext()) { diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java index 5250a660d..844b6c364 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java @@ -60,39 +60,39 @@ public class HiveSyncToolTest { @Test public void testSchemaConvertArray() throws IOException { // Testing the 3-level annotation structure - MessageType schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().optional(PrimitiveType.PrimitiveTypeName.INT32).named("element") - .named("list").named("int_list").named("ArrayOfInts"); + MessageType schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup() + .optional(PrimitiveType.PrimitiveTypeName.INT32).named("element") + .named("list").named("int_list").named("ArrayOfInts"); String schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`int_list` ARRAY< int>", schemaString); // A array of arrays - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().requiredGroup().as(OriginalType.LIST).repeatedGroup() - .required(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("list") - .named("element").named("list").named("int_list_list").named("ArrayOfArrayOfInts"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup().requiredGroup() + .as(OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("list") + .named("element").named("list").named("int_list_list").named("ArrayOfArrayOfInts"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`int_list_list` ARRAY< ARRAY< int>>", schemaString); // A list of integers - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeated(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("int_list") - .named("ArrayOfInts"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST) + .repeated(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("int_list") + .named("ArrayOfInts"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`int_list` ARRAY< int>", schemaString); // A list of structs with two fields - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") - .required(PrimitiveType.PrimitiveTypeName.INT32).named("num").named("element") - .named("tuple_list").named("ArrayOfTuples"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") + .required(PrimitiveType.PrimitiveTypeName.INT32).named("num").named("element") + .named("tuple_list").named("ArrayOfTuples"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`tuple_list` ARRAY< STRUCT< `str` : binary, `num` : int>>", schemaString); @@ -100,10 +100,10 @@ public class HiveSyncToolTest { // A list of structs with a single field // For this case, since the inner group name is "array", we treat the // element type as a one-element struct. - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") - .named("array").named("one_tuple_list").named("ArrayOfOneTuples"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("array") + .named("one_tuple_list").named("ArrayOfOneTuples"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString); @@ -111,10 +111,10 @@ public class HiveSyncToolTest { // A list of structs with a single field // For this case, since the inner group name ends with "_tuple", we also treat the // element type as a one-element struct. - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") - .named("one_tuple_list_tuple").named("one_tuple_list").named("ArrayOfOneTuples2"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") + .named("one_tuple_list_tuple").named("one_tuple_list").named("ArrayOfOneTuples2"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString); @@ -122,22 +122,22 @@ public class HiveSyncToolTest { // A list of structs with a single field // Unlike the above two cases, for this the element type is the type of the // only field in the struct. - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") - .named("one_tuple_list").named("one_tuple_list").named("ArrayOfOneTuples3"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str") + .named("one_tuple_list").named("one_tuple_list").named("ArrayOfOneTuples3"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`one_tuple_list` ARRAY< binary>", schemaString); // A list of maps - schema = - parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.OriginalType.LIST) - .repeatedGroup().as(OriginalType.MAP).repeatedGroup().as(OriginalType.MAP_KEY_VALUE) - .required(PrimitiveType.PrimitiveTypeName.BINARY).as(OriginalType.UTF8) - .named("string_key").required(PrimitiveType.PrimitiveTypeName.INT32) - .named("int_value").named("key_value").named("array").named("map_list") - .named("ArrayOfMaps"); + schema = parquet.schema.Types.buildMessage().optionalGroup() + .as(parquet.schema.OriginalType.LIST).repeatedGroup().as(OriginalType.MAP) + .repeatedGroup().as(OriginalType.MAP_KEY_VALUE) + .required(PrimitiveType.PrimitiveTypeName.BINARY).as(OriginalType.UTF8) + .named("string_key").required(PrimitiveType.PrimitiveTypeName.INT32) + .named("int_value").named("key_value").named("array").named("map_list") + .named("ArrayOfMaps"); schemaString = SchemaUtil.generateSchemaString(schema); assertEquals("`map_list` ARRAY< MAP< string, int>>", schemaString); @@ -146,7 +146,8 @@ public class HiveSyncToolTest { @Test public void testBasicSync() - throws IOException, InitializationError, URISyntaxException, TException, InterruptedException { + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { String commitTime = "100"; TestUtil.createCOWDataset(commitTime, 5); HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, @@ -160,18 +161,17 @@ public class HiveSyncToolTest { assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + " should exist after sync completes", hiveClient.doesTableExist()); assertEquals("Hive Schema should match the dataset schema + partition field", - hiveClient.getTableSchema().size(), - hiveClient.getDataSchema().getColumns().size() + 1); + hiveClient.getTableSchema().size(), hiveClient.getDataSchema().getColumns().size() + 1); assertEquals("Table partitions should match the number of partitions we wrote", 5, hiveClient.scanTablePartitions().size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", - commitTime, - hiveClient.getLastCommitTimeSynced().get()); + commitTime, hiveClient.getLastCommitTimeSynced().get()); } @Test public void testSyncIncremental() - throws IOException, InitializationError, URISyntaxException, TException, InterruptedException { + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { String commitTime1 = "100"; TestUtil.createCOWDataset(commitTime1, 5); HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, @@ -183,8 +183,7 @@ public class HiveSyncToolTest { assertEquals("Table partitions should match the number of partitions we wrote", 5, hiveClient.scanTablePartitions().size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", - commitTime1, - hiveClient.getLastCommitTimeSynced().get()); + commitTime1, hiveClient.getLastCommitTimeSynced().get()); // Now lets create more parititions and these are the only ones which needs to be synced DateTime dateTime = DateTime.now().plusDays(6); @@ -192,33 +191,32 @@ public class HiveSyncToolTest { TestUtil.addCOWPartitions(1, true, dateTime, commitTime2); // Lets do the sync - hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, - TestUtil.getHiveConf(), TestUtil.fileSystem); - List writtenPartitionsSince = hiveClient - .getPartitionsWrittenToSince(Optional.of(commitTime1)); + hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), + TestUtil.fileSystem); + List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince( + Optional.of(commitTime1)); assertEquals("We should have one partition written after 100 commit", 1, writtenPartitionsSince.size()); List hivePartitions = hiveClient.scanTablePartitions(); - List partitionEvents = hiveClient - .getPartitionEvents(hivePartitions, writtenPartitionsSince); + List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, + writtenPartitionsSince); assertEquals("There should be only one paritition event", 1, partitionEvents.size()); assertEquals("The one partition event must of type ADD", PartitionEventType.ADD, partitionEvents.iterator().next().eventType); - tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), - TestUtil.fileSystem); + tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); // Sync should add the one partition assertEquals("The one partition we wrote should be added to hive", 6, hiveClient.scanTablePartitions().size()); - assertEquals("The last commit that was sycned should be 101", - commitTime2, + assertEquals("The last commit that was sycned should be 101", commitTime2, hiveClient.getLastCommitTimeSynced().get()); } @Test public void testSyncIncrementalWithSchemaEvolution() - throws IOException, InitializationError, URISyntaxException, TException, InterruptedException { + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { String commitTime1 = "100"; TestUtil.createCOWDataset(commitTime1, 5); HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, @@ -236,30 +234,27 @@ public class HiveSyncToolTest { TestUtil.addCOWPartitions(1, false, dateTime, commitTime2); // Lets do the sync - tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), - TestUtil.fileSystem); + tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); - assertEquals("Hive Schema has evolved and should not be 3 more field", - fields + 3, + assertEquals("Hive Schema has evolved and should not be 3 more field", fields + 3, hiveClient.getTableSchema().size()); assertEquals("Hive Schema has evolved - Field favorite_number has evolved from int to long", - "BIGINT", - hiveClient.getTableSchema().get("favorite_number")); + "BIGINT", hiveClient.getTableSchema().get("favorite_number")); assertTrue("Hive Schema has evolved - Field favorite_movie was added", hiveClient.getTableSchema().containsKey("favorite_movie")); // Sync should add the one partition assertEquals("The one partition we wrote should be added to hive", 6, hiveClient.scanTablePartitions().size()); - assertEquals("The last commit that was sycned should be 101", - commitTime2, + assertEquals("The last commit that was sycned should be 101", commitTime2, hiveClient.getLastCommitTimeSynced().get()); } @Test public void testSyncMergeOnRead() - throws IOException, InitializationError, URISyntaxException, TException, InterruptedException { + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { String commitTime = "100"; String deltaCommitTime = "101"; TestUtil.createMORDataset(commitTime, deltaCommitTime, 5); @@ -280,8 +275,7 @@ public class HiveSyncToolTest { assertEquals("Table partitions should match the number of partitions we wrote", 5, hiveClient.scanTablePartitions().size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", - deltaCommitTime, - hiveClient.getLastCommitTimeSynced().get()); + deltaCommitTime, hiveClient.getLastCommitTimeSynced().get()); // Now lets create more parititions and these are the only ones which needs to be synced DateTime dateTime = DateTime.now().plusDays(6); @@ -291,11 +285,10 @@ public class HiveSyncToolTest { TestUtil.addCOWPartitions(1, true, dateTime, commitTime2); TestUtil.addMORPartitions(1, true, false, dateTime, commitTime2, deltaCommitTime2); // Lets do the sync - tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), - TestUtil.fileSystem); + tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); - hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, - TestUtil.getHiveConf(), TestUtil.fileSystem); + hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), + TestUtil.fileSystem); assertEquals("Hive Schema should match the evolved dataset schema + partition field", hiveClient.getTableSchema().size(), @@ -303,14 +296,14 @@ public class HiveSyncToolTest { // Sync should add the one partition assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClient.scanTablePartitions().size()); - assertEquals("The last commit that was sycned should be 103", - deltaCommitTime2, + assertEquals("The last commit that was sycned should be 103", deltaCommitTime2, hiveClient.getLastCommitTimeSynced().get()); } @Test public void testSyncMergeOnReadRT() - throws IOException, InitializationError, URISyntaxException, TException, InterruptedException { + throws IOException, InitializationError, URISyntaxException, TException, + InterruptedException { String commitTime = "100"; String deltaCommitTime = "101"; String roTablename = TestUtil.hiveSyncConfig.tableName; @@ -321,8 +314,7 @@ public class HiveSyncToolTest { TestUtil.getHiveConf(), TestUtil.fileSystem); assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE - + " should not exist initially", - hiveClientRT.doesTableExist()); + + " should not exist initially", hiveClientRT.doesTableExist()); // Lets do the sync HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), @@ -330,8 +322,7 @@ public class HiveSyncToolTest { tool.syncHoodieTable(); assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE - + " should exist after sync completes", - hiveClientRT.doesTableExist()); + + " should exist after sync completes", hiveClientRT.doesTableExist()); assertEquals("Hive Schema should match the dataset schema + partition field", hiveClientRT.getTableSchema().size(), @@ -339,8 +330,7 @@ public class HiveSyncToolTest { assertEquals("Table partitions should match the number of partitions we wrote", 5, hiveClientRT.scanTablePartitions().size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", - deltaCommitTime, - hiveClientRT.getLastCommitTimeSynced().get()); + deltaCommitTime, hiveClientRT.getLastCommitTimeSynced().get()); // Now lets create more parititions and these are the only ones which needs to be synced DateTime dateTime = DateTime.now().plusDays(6); @@ -350,11 +340,10 @@ public class HiveSyncToolTest { TestUtil.addCOWPartitions(1, true, dateTime, commitTime2); TestUtil.addMORPartitions(1, true, false, dateTime, commitTime2, deltaCommitTime2); // Lets do the sync - tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), - TestUtil.fileSystem); + tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); - hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig, - TestUtil.getHiveConf(), TestUtil.fileSystem); + hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), + TestUtil.fileSystem); assertEquals("Hive Schema should match the evolved dataset schema + partition field", hiveClientRT.getTableSchema().size(), @@ -362,8 +351,7 @@ public class HiveSyncToolTest { // Sync should add the one partition assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClientRT.scanTablePartitions().size()); - assertEquals("The last commit that was sycned should be 103", - deltaCommitTime2, + assertEquals("The last commit that was sycned should be 103", deltaCommitTime2, hiveClientRT.getLastCommitTimeSynced().get()); TestUtil.hiveSyncConfig.tableName = roTablename; } diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java index ff260f893..a2a3ab2bb 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java @@ -16,6 +16,9 @@ package com.uber.hoodie.hive; +import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_TASK_PARTITIONID; +import static org.junit.Assert.fail; + import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -39,6 +42,15 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.hive.util.HiveTestService; +import java.io.File; +import java.io.IOException; +import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.UUID; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.commons.io.FileUtils; @@ -58,19 +70,6 @@ import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; import org.junit.runners.model.InitializationError; -import java.io.File; -import java.io.IOException; -import java.net.URISyntaxException; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.UUID; - -import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_TASK_PARTITIONID; -import static org.junit.Assert.fail; - @SuppressWarnings("SameParameterValue") public class TestUtil { @@ -127,8 +126,7 @@ public class TestUtil { client.updateHiveSQL("drop table if exists " + tableName); } createdTablesSet.clear(); - client.updateHiveSQL( - "drop database if exists " + hiveSyncConfig.databaseName); + client.updateHiveSQL("drop database if exists " + hiveSyncConfig.databaseName); client.updateHiveSQL("create database " + hiveSyncConfig.databaseName); } @@ -182,9 +180,8 @@ public class TestUtil { createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); - commitMetadata.getPartitionToWriteStats() - .forEach((key, value) -> value.stream() - .forEach(l -> compactionMetadata.addWriteStat(key, l))); + commitMetadata.getPartitionToWriteStats().forEach( + (key, value) -> value.stream().forEach(l -> compactionMetadata.addWriteStat(key, l))); createCompactionCommitFile(compactionMetadata, commitTime); // Write a delta commit HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), @@ -202,8 +199,7 @@ public class TestUtil { } static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, - boolean isLogSchemaSimple, DateTime startFrom, - String commitTime, String deltaCommitTime) + boolean isLogSchemaSimple, DateTime startFrom, String commitTime, String deltaCommitTime) throws IOException, URISyntaxException, InterruptedException { HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, isParquetSchemaSimple, startFrom, commitTime); @@ -211,9 +207,8 @@ public class TestUtil { createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); - commitMetadata.getPartitionToWriteStats() - .forEach((key, value) -> value.stream() - .forEach(l -> compactionMetadata.addWriteStat(key, l))); + commitMetadata.getPartitionToWriteStats().forEach( + (key, value) -> value.stream().forEach(l -> compactionMetadata.addWriteStat(key, l))); createCompactionCommitFile(compactionMetadata, commitTime); HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), isLogSchemaSimple); @@ -264,8 +259,8 @@ public class TestUtil { for (int i = 0; i < 5; i++) { // Create 5 files String fileId = UUID.randomUUID().toString(); - Path filePath = new Path(partPath.toString() + "/" + FSUtils - .makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileId)); + Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(commitTime, + DEFAULT_TASK_PARTITIONID, fileId)); generateParquetData(filePath, isParquetSchemaSimple); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setFileId(fileId); @@ -283,11 +278,10 @@ public class TestUtil { org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema); BloomFilter filter = new BloomFilter(1000, 0.0001); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, filter); - ParquetWriter writer = new ParquetWriter(filePath, - writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, - ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, - ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, - fileSystem.getConf()); + ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, + 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, + ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, + ParquetWriter.DEFAULT_WRITER_VERSION, fileSystem.getConf()); List testRecords = (isParquetSchemaSimple ? SchemaTestUtil .generateTestRecords(0, 100) @@ -309,10 +303,10 @@ public class TestUtil { HoodieDataFile dataFile = new HoodieDataFile(fileSystem.getFileStatus(parquetFilePath)); // Write a log file for this parquet file Writer logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(parquetFilePath.getParent()) - .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(dataFile.getFileId()) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId(dataFile.getFileId()) .overBaseCommit(dataFile.getCommitTime()).withFs(fileSystem).build(); - List records = (isLogSchemaSimple ? SchemaTestUtil - .generateTestRecords(0, 100) + List records = (isLogSchemaSimple ? SchemaTestUtil.generateTestRecords(0, 100) : SchemaTestUtil.generateEvolvedTestRecords(100, 100)); Map header = Maps.newHashMap(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime()); @@ -329,37 +323,37 @@ public class TestUtil { } } - private static void createCommitFile( - HoodieCommitMetadata commitMetadata, String commitTime) + private static void createCommitFile(HoodieCommitMetadata commitMetadata, String commitTime) throws IOException { byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); Path fullPath = new Path( hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCommitFileName(commitTime)); + .makeCommitFileName( + commitTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); fsout.close(); } - private static void createCompactionCommitFile( - HoodieCommitMetadata commitMetadata, String commitTime) - throws IOException { + private static void createCompactionCommitFile(HoodieCommitMetadata commitMetadata, + String commitTime) throws IOException { byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); Path fullPath = new Path( hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeCommitFileName(commitTime)); + .makeCommitFileName( + commitTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); fsout.close(); } - private static void createDeltaCommitFile( - HoodieCommitMetadata deltaCommitMetadata, String deltaCommitTime) - throws IOException { + private static void createDeltaCommitFile(HoodieCommitMetadata deltaCommitMetadata, + String deltaCommitTime) throws IOException { byte[] bytes = deltaCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); Path fullPath = new Path( hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline - .makeDeltaFileName(deltaCommitTime)); + .makeDeltaFileName( + deltaCommitTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); fsout.close(); diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java index 7b9172ead..ec9cd3ca2 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java @@ -16,7 +16,6 @@ package com.uber.hoodie.hive.util; - import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.google.common.io.Files; @@ -84,8 +83,7 @@ public class HiveTestService { } public HiveServer2 start() throws IOException { - Preconditions - .checkState(workDir != null, "The work dir must be set before starting cluster."); + Preconditions.checkState(workDir != null, "The work dir must be set before starting cluster."); if (hadoopConf == null) { hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); @@ -93,8 +91,7 @@ public class HiveTestService { String localHiveLocation = getHiveLocation(workDir); if (clean) { - LOG.info( - "Cleaning Hive cluster data at: " + localHiveLocation + " and starting fresh."); + LOG.info("Cleaning Hive cluster data at: " + localHiveLocation + " and starting fresh."); File file = new File(localHiveLocation); FileUtils.deleteDirectory(file); } @@ -134,11 +131,9 @@ public class HiveTestService { hadoopConf = null; } - private HiveConf configureHive(Configuration conf, String localHiveLocation) - throws IOException { + private HiveConf configureHive(Configuration conf, String localHiveLocation) throws IOException { conf.set("hive.metastore.local", "false"); - conf.set(HiveConf.ConfVars.METASTOREURIS.varname, - "thrift://" + bindIP + ":" + metastorePort); + conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://" + bindIP + ":" + metastorePort); conf.set(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, bindIP); conf.setInt(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, serverPort); // The following line to turn of SASL has no effect since HiveAuthFactory calls @@ -154,8 +149,7 @@ public class HiveTestService { File derbyLogFile = new File(localHiveDir, "derby.log"); derbyLogFile.createNewFile(); setSystemProperty("derby.stream.error.file", derbyLogFile.getPath()); - conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, - Files.createTempDir().getAbsolutePath()); + conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, Files.createTempDir().getAbsolutePath()); return new HiveConf(conf, this.getClass()); } @@ -269,8 +263,8 @@ public class HiveTestService { int minWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMINTHREADS); int maxWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXTHREADS); boolean tcpKeepAlive = conf.getBoolVar(HiveConf.ConfVars.METASTORE_TCP_KEEP_ALIVE); - boolean useFramedTransport = - conf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_FRAMED_TRANSPORT); + boolean useFramedTransport = conf.getBoolVar( + HiveConf.ConfVars.METASTORE_USE_THRIFT_FRAMED_TRANSPORT); // don't support SASL yet //boolean useSasl = conf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL); @@ -282,21 +276,20 @@ public class HiveTestService { tcpKeepAlive ? new TServerSocketKeepAlive(address) : new TServerSocket(address); } else { - serverTransport = - tcpKeepAlive ? new TServerSocketKeepAlive(port) : new TServerSocket(port); + serverTransport = tcpKeepAlive ? new TServerSocketKeepAlive(port) : new TServerSocket(port); } TProcessor processor; TTransportFactory transFactory; IHMSHandler handler = (IHMSHandler) HiveMetaStore - .newRetryingHMSHandler("new db based metaserver", conf, true); + .newRetryingHMSHandler("new db based metaserver", + conf, true); if (conf.getBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI)) { - transFactory = useFramedTransport ? - new ChainedTTransportFactory(new TFramedTransport.Factory(), - new TUGIContainingTransport.Factory()) : - new TUGIContainingTransport.Factory(); + transFactory = + useFramedTransport ? new ChainedTTransportFactory(new TFramedTransport.Factory(), + new TUGIContainingTransport.Factory()) : new TUGIContainingTransport.Factory(); processor = new TUGIBasedProcessor(handler); LOG.info("Starting DB backed MetaStore Server with SetUGI enabled"); @@ -307,10 +300,11 @@ public class HiveTestService { LOG.info("Starting DB backed MetaStore Server"); } - TThreadPoolServer.Args args = - new TThreadPoolServer.Args(serverTransport).processor(processor) - .transportFactory(transFactory).protocolFactory(new TBinaryProtocol.Factory()) - .minWorkerThreads(minWorkerThreads).maxWorkerThreads(maxWorkerThreads); + TThreadPoolServer.Args args = new TThreadPoolServer.Args(serverTransport).processor(processor) + .transportFactory(transFactory) + .protocolFactory(new TBinaryProtocol.Factory()) + .minWorkerThreads(minWorkerThreads) + .maxWorkerThreads(maxWorkerThreads); final TServer tServer = new TThreadPoolServer(args); executorService.submit(new Runnable() { diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java b/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java index 26e9cd31e..8e452c8dc 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/BaseAvroPayload.java @@ -38,7 +38,6 @@ public abstract class BaseAvroPayload implements Serializable { protected final Comparable orderingVal; /** - * * @param record * @param orderingVal */ diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java index b16202cff..19226af91 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java @@ -87,8 +87,8 @@ public class DataSourceUtils { public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record, Comparable orderingVal) throws IOException { try { - return (HoodieRecordPayload) ConstructorUtils - .invokeConstructor(Class.forName(payloadClass), (Object) record, (Object) orderingVal); + return (HoodieRecordPayload) ConstructorUtils.invokeConstructor(Class.forName(payloadClass), + (Object) record, (Object) orderingVal); } catch (Throwable e) { throw new IOException("Could not create payload for class: " + payloadClass, e); } @@ -103,36 +103,26 @@ public class DataSourceUtils { }); } - public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, - String schemaStr, - String basePath, - String tblName, - Map parameters) throws Exception { - HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() - .combineInput(true, true) - .withPath(basePath) - .withAutoCommit(false) - .withSchema(schemaStr) - .forTable(tblName) - .withIndexConfig( - HoodieIndexConfig.newBuilder() - .withIndexType(HoodieIndex.IndexType.BLOOM) - .build()) + public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, + String basePath, String tblName, Map parameters) throws Exception { + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().combineInput(true, true) + .withPath(basePath).withAutoCommit(false) + .withSchema(schemaStr).forTable(tblName).withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY())) + .withPayloadClass(parameters.get( + DataSourceWriteOptions + .PAYLOAD_CLASS_OPT_KEY())) .build()) // override above with Hoodie configs specified as options. - .withProps(parameters) - .build(); + .withProps(parameters).build(); return new HoodieWriteClient<>(jssc, writeConfig); } public static JavaRDD doWriteOperation(HoodieWriteClient client, - JavaRDD hoodieRecords, - String commitTime, - String operation) { + JavaRDD hoodieRecords, String commitTime, String operation) { if (operation.equals(DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL())) { return client.bulkInsert(hoodieRecords, commitTime); } else if (operation.equals(DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())) { @@ -143,14 +133,9 @@ public class DataSourceUtils { } } - public static HoodieRecord createHoodieRecord(GenericRecord gr, - Comparable orderingVal, - HoodieKey hKey, - String payloadClass) throws IOException { - HoodieRecordPayload payload = DataSourceUtils.createPayload( - payloadClass, - gr, - orderingVal); + public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal, + HoodieKey hKey, String payloadClass) throws IOException { + HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal); return new HoodieRecord<>(hKey, payload); } } 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 996786a7d..f3fe94100 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java @@ -67,11 +67,12 @@ public class HoodieDataSourceHelpers { */ public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) { HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null); + .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), + null); if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) { return table.getActiveTimeline().getTimelineOfActions( Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, - HoodieActiveTimeline.DELTA_COMMIT_ACTION)); + HoodieActiveTimeline.DELTA_COMMIT_ACTION)); } else { return table.getCommitTimeline().filterCompletedInstants(); } diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java b/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java index f23148ad3..74aa74560 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/KeyGenerator.java @@ -24,7 +24,8 @@ import org.apache.avro.generic.GenericRecord; import org.apache.commons.configuration.PropertiesConfiguration; /** - * Abstract class to extend for plugging in extraction of {@link com.uber.hoodie.common.model.HoodieKey} + * Abstract class to extend for plugging in extraction of + * {@link com.uber.hoodie.common.model.HoodieKey} * from an Avro record */ public abstract class KeyGenerator implements Serializable { diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java b/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java index 6b12b1f97..48396f08c 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java @@ -28,7 +28,7 @@ import org.apache.avro.generic.IndexedRecord; /** * Default payload used for delta streamer. - * + *

* 1. preCombine - Picks the latest delta record for a key, based on an ordering field 2. * combineAndGetUpdateValue/getInsertValue - Simply overwrites storage with latest delta record */ @@ -36,7 +36,6 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements HoodieRecordPayload { /** - * * @param record * @param orderingVal */ diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala index 684024887..4312636fa 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/AvroConversionUtils.scala @@ -47,10 +47,11 @@ object AvroConversionUtils { structName: String, recordNamespace: String): (Any) => Any = { dataType match { - case BinaryType => (item: Any) => item match { - case null => null - case bytes: Array[Byte] => ByteBuffer.wrap(bytes) - } + case BinaryType => (item: Any) => + item match { + case null => null + case bytes: Array[Byte] => ByteBuffer.wrap(bytes) + } case ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | StringType | BooleanType => identity case _: DecimalType => (item: Any) => if (item == null) null else item.toString diff --git a/hoodie-spark/src/test/java/DataSourceTestUtils.java b/hoodie-spark/src/test/java/DataSourceTestUtils.java index a39c42ac3..e3d4c65ee 100644 --- a/hoodie-spark/src/test/java/DataSourceTestUtils.java +++ b/hoodie-spark/src/test/java/DataSourceTestUtils.java @@ -32,17 +32,15 @@ public class DataSourceTestUtils { try { String str = ((TestRawTripPayload) record.getData()).getJsonData(); str = "{" + str.substring(str.indexOf("\"timestamp\":")); - return Optional - .of(str.replaceAll("}", ", \"partition\": \"" + record.getPartitionPath() + "\"}")); + return Optional.of(str.replaceAll("}", + ", \"partition\": \"" + record.getPartitionPath() + "\"}")); } catch (IOException e) { return Optional.empty(); } } public static List convertToStringList(List records) { - return records.stream().map(hr -> convertToString(hr)) - .filter(os -> os.isPresent()) - .map(os -> os.get()) - .collect(Collectors.toList()); + return records.stream().map(hr -> convertToString(hr)).filter(os -> os.isPresent()) + .map(os -> os.get()).collect(Collectors.toList()); } } diff --git a/hoodie-spark/src/test/java/HoodieJavaApp.java b/hoodie-spark/src/test/java/HoodieJavaApp.java index ff2a03416..bef67cfc8 100644 --- a/hoodie-spark/src/test/java/HoodieJavaApp.java +++ b/hoodie-spark/src/test/java/HoodieJavaApp.java @@ -1,12 +1,12 @@ /* * Copyright (c) 2017 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. @@ -16,7 +16,6 @@ * */ - import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import com.uber.hoodie.DataSourceReadOptions; @@ -68,10 +67,9 @@ public class HoodieJavaApp { public void run() throws Exception { // Spark session setup.. - SparkSession spark = SparkSession.builder() - .appName("Hoodie Spark APP") - .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .master("local[1]") + SparkSession spark = SparkSession.builder().appName("Hoodie Spark APP") + .config("spark.serializer", + "org.apache.spark.serializer.KryoSerializer").master("local[1]") .getOrCreate(); JavaSparkContext jssc = new JavaSparkContext(spark.sparkContext()); FileSystem fs = FileSystem.get(jssc.hadoopConfiguration()); @@ -83,13 +81,12 @@ public class HoodieJavaApp { * Commit with only inserts */ // Generate some input.. - List records1 = DataSourceTestUtils - .convertToStringList(dataGen.generateInserts("001"/* ignore */, 100)); + List records1 = DataSourceTestUtils.convertToStringList( + dataGen.generateInserts("001"/* ignore */, 100)); Dataset inputDF1 = spark.read().json(jssc.parallelize(records1, 2)); // Save as hoodie dataset (copy on write) - inputDF1.write() - .format("com.uber.hoodie") // specify the hoodie source + inputDF1.write().format("com.uber.hoodie") // specify the hoodie source .option("hoodie.insert.shuffle.parallelism", "2") // any hoodie client config can be passed like this .option("hoodie.upsert.shuffle.parallelism", @@ -104,7 +101,8 @@ public class HoodieJavaApp { "timestamp") // use to combine duplicate records in input/with disk val .option(HoodieWriteConfig.TABLE_NAME, tableName) // Used by hive sync and queries .mode( - SaveMode.Overwrite) // This will remove any existing data at path below, and create a new dataset if needed + SaveMode.Overwrite) // This will remove any existing data at path below, and create a + // new dataset if needed .save(tablePath); // ultimately where the dataset will be placed String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath); logger.info("First commit at instant time :" + commitInstantTime1); @@ -112,28 +110,24 @@ public class HoodieJavaApp { /** * Commit that updates records */ - List records2 = DataSourceTestUtils - .convertToStringList(dataGen.generateUpdates("002"/* ignore */, 100)); + List records2 = DataSourceTestUtils.convertToStringList( + dataGen.generateUpdates("002"/* ignore */, 100)); Dataset inputDF2 = spark.read().json(jssc.parallelize(records2, 2)); - inputDF2.write() - .format("com.uber.hoodie") - .option("hoodie.insert.shuffle.parallelism", "2") + inputDF2.write().format("com.uber.hoodie").option("hoodie.insert.shuffle.parallelism", "2") .option("hoodie.upsert.shuffle.parallelism", "2") .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") - .option(HoodieWriteConfig.TABLE_NAME, tableName) - .mode(SaveMode.Append) - .save(tablePath); + .option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append).save(tablePath); String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath); logger.info("Second commit at instant time :" + commitInstantTime1); /** * Read & do some queries */ - Dataset hoodieROViewDF = spark.read() - .format("com.uber.hoodie") - // pass any path glob, can include hoodie & non-hoodie datasets + Dataset hoodieROViewDF = spark.read().format("com.uber.hoodie") + // pass any path glob, can include hoodie & non-hoodie + // datasets .load(tablePath + "/*/*/*/*"); hoodieROViewDF.registerTempTable("hoodie_ro"); spark.sql("describe hoodie_ro").show(); @@ -149,7 +143,8 @@ public class HoodieJavaApp { DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL()) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1) // Only changes in write 2 above - .load(tablePath); // For incremental view, pass in the root/base path of dataset + .load( + tablePath); // For incremental view, pass in the root/base path of dataset logger.info("You will only see records from : " + commitInstantTime2); hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show(); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java index 77184971c..70871b30a 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java @@ -65,8 +65,7 @@ public class HDFSParquetImporter implements Serializable { private transient FileSystem fs; public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd"); - public HDFSParquetImporter( - Config cfg) throws IOException { + public HDFSParquetImporter(Config cfg) throws IOException { this.cfg = cfg; } @@ -77,9 +76,8 @@ public class HDFSParquetImporter implements Serializable { @Override public void validate(String name, String value) throws ParameterException { if (value == null || !validFormats.contains(value)) { - throw new ParameterException(String - .format("Invalid format type: value:%s: supported formats:%s", value, - validFormats)); + throw new ParameterException(String.format( + "Invalid format type: value:%s: supported formats:%s", value, validFormats)); } } } @@ -91,9 +89,8 @@ public class HDFSParquetImporter implements Serializable { @Override public void validate(String name, String value) throws ParameterException { if (value == null || !validSourceTypes.contains(value)) { - throw new ParameterException(String - .format("Invalid source type: value:%s: supported source types:%s", value, - validSourceTypes)); + throw new ParameterException(String.format( + "Invalid source type: value:%s: supported source types:%s", value, validSourceTypes)); } } } @@ -127,23 +124,21 @@ public class HDFSParquetImporter implements Serializable { "-sf"}, description = "path for Avro schema file", required = true) public String schemaFile = null; @Parameter(names = {"--format", - "-f"}, description = "Format for the input data.", required = false, - validateValueWith = FormatValidator.class) + "-f"}, description = "Format for the input data.", required = false, validateValueWith = + FormatValidator.class) public String format = null; - @Parameter(names = {"--spark-master", - "-ms"}, description = "Spark master", required = false) + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false) public String sparkMaster = null; @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true) public String sparkMemory = null; - @Parameter(names = {"--retry", - "-rt"}, description = "number of retries", required = false) + @Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false) public int retry = 0; @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; } - public static void main(String args[]) throws Exception { + public static void main(String[] args) throws Exception { final HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config(); JCommander cmd = new JCommander(cfg, args); if (cfg.help || args.length == 0) { @@ -182,8 +177,7 @@ public class HDFSParquetImporter implements Serializable { // Read schema file. Path p = new Path(cfg.schemaFile); if (!fs.exists(p)) { - throw new Exception( - String.format("Could not find - %s - schema file.", cfg.schemaFile)); + throw new Exception(String.format("Could not find - %s - schema file.", cfg.schemaFile)); } long len = fs.getFileStatus(p).getLen(); ByteBuffer buf = ByteBuffer.allocate((int) len); @@ -205,8 +199,7 @@ public class HDFSParquetImporter implements Serializable { try { // Verify that targetPath is not present. if (fs.exists(new Path(cfg.targetPath))) { - throw new HoodieIOException( - String.format("Make sure %s is not present.", cfg.targetPath)); + throw new HoodieIOException(String.format("Make sure %s is not present.", cfg.targetPath)); } do { ret = dataImport(jsc); @@ -232,7 +225,8 @@ public class HDFSParquetImporter implements Serializable { Properties properties = new Properties(); properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName); properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType); - HoodieTableMetaClient.initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties); + HoodieTableMetaClient + .initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties); HoodieWriteClient client = createHoodieClient(jsc, cfg.targetPath, schemaStr, cfg.parallelism); @@ -240,50 +234,54 @@ public class HDFSParquetImporter implements Serializable { Job job = Job.getInstance(jsc.hadoopConfiguration()); // To parallelize reading file status. job.getConfiguration().set(FileInputFormat.LIST_STATUS_NUM_THREADS, "1024"); - AvroReadSupport.setAvroReadSchema(jsc.hadoopConfiguration(), - (new Schema.Parser().parse(schemaStr))); + AvroReadSupport + .setAvroReadSchema(jsc.hadoopConfiguration(), (new Schema.Parser().parse(schemaStr))); ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class)); - JavaRDD> hoodieRecords = jsc - .newAPIHadoopFile(cfg.srcPath, ParquetInputFormat.class, Void.class, - GenericRecord.class, job.getConfiguration()) - // To reduce large number of tasks. + JavaRDD> hoodieRecords = jsc.newAPIHadoopFile(cfg.srcPath, + ParquetInputFormat.class, Void.class, GenericRecord.class, job.getConfiguration()) + // To reduce large number of + // tasks. .coalesce(16 * cfg.parallelism) .map(entry -> { - GenericRecord genericRecord = ((Tuple2) entry)._2(); - Object partitionField = genericRecord.get(cfg.partitionKey); - if (partitionField == null) { - throw new HoodieIOException( - "partition key is missing. :" + cfg.partitionKey); - } - Object rowField = genericRecord.get(cfg.rowKey); - if (rowField == null) { - throw new HoodieIOException( - "row field is missing. :" + cfg.rowKey); - } - long ts = (long) ((Double) partitionField * 1000l); - String partitionPath = PARTITION_FORMATTER.format(new Date(ts)); - return new HoodieRecord<>( - new HoodieKey((String) rowField, partitionPath), - new HoodieJsonPayload(genericRecord.toString())); - } - ); + GenericRecord genericRecord + = ((Tuple2) entry)._2(); + Object partitionField = + genericRecord.get(cfg.partitionKey); + if (partitionField == null) { + throw new HoodieIOException( + "partition key is missing. :" + + cfg.partitionKey); + } + Object rowField = genericRecord.get(cfg.rowKey); + if (rowField == null) { + throw new HoodieIOException( + "row field is missing. :" + cfg.rowKey); + } + long ts = (long) ((Double) partitionField * 1000L); + String partitionPath = + PARTITION_FORMATTER.format(new Date(ts)); + return new HoodieRecord<>( + new HoodieKey( + (String) rowField, partitionPath), + new HoodieJsonPayload( + genericRecord.toString())); + }); // Get commit time. String commitTime = client.startCommit(); JavaRDD writeResponse = client.bulkInsert(hoodieRecords, commitTime); Accumulator errors = jsc.accumulator(0); writeResponse.foreach(writeStatus -> { - if (writeStatus.hasErrors()) { - errors.add(1); - logger.error(String.format("Error processing records :writeStatus:%s", - writeStatus.getStat().toString())); - } + if (writeStatus.hasErrors()) { + errors.add(1); + logger.error(String.format("Error processing records :writeStatus:%s", + writeStatus.getStat().toString())); + } }); if (errors.value() == 0) { - logger.info(String - .format("Dataset imported into hoodie dataset with %s commit time.", - commitTime)); + logger.info( + String.format("Dataset imported into hoodie dataset with %s commit time.", commitTime)); return 0; } logger.error(String.format("Import failed with %d errors.", errors.value())); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java index a6fbc3f36..07d3ecbe9 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java @@ -52,9 +52,9 @@ import org.stringtemplate.v4.ST; /** * Utility to pull data after a given commit, based on the supplied HiveQL and save the delta as * another hive temporary table. - * + *

* Current Limitations: - * + *

* - Only the source table can be incrementally pulled (usually the largest table) - The * incrementally pulled table can't be referenced more than once. */ @@ -66,8 +66,7 @@ public class HiveIncrementalPuller { public static class Config implements Serializable { @Parameter(names = {"--hiveUrl"}) - public String hiveJDBCUrl = - "jdbc:hive2://localhost:10014/;transportMode=http;httpPath=hs2"; + public String hiveJDBCUrl = "jdbc:hive2://localhost:10014/;transportMode=http;httpPath=hs2"; @Parameter(names = {"--hiveUser"}) public String hiveUsername = "hive"; @Parameter(names = {"--hivePass"}) @@ -113,8 +112,8 @@ public class HiveIncrementalPuller { public HiveIncrementalPuller(Config config) throws IOException { this.config = config; validateConfig(config); - String templateContent = IOUtils - .toString(this.getClass().getResourceAsStream("IncrementalPull.sqltemplate")); + String templateContent = IOUtils.toString( + this.getClass().getResourceAsStream("IncrementalPull.sqltemplate")); incrementalPullSQLtemplate = new ST(templateContent); } @@ -182,19 +181,22 @@ public class HiveIncrementalPuller { String storedAsClause = getStoredAsClause(); incrementalPullSQLtemplate.add("storedAsClause", storedAsClause); - String incrementalSQL = - new Scanner(new File(config.incrementalSQLFile)).useDelimiter("\\Z").next(); + String incrementalSQL = new Scanner(new File(config.incrementalSQLFile)).useDelimiter("\\Z") + .next(); if (!incrementalSQL.contains(config.sourceDb + "." + config.sourceTable)) { log.info("Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable - + ", which means its pulling from a different table. Fencing this from happening."); + + ", which means its pulling from a different table. Fencing this from " + + "happening."); throw new HoodieIncrementalPullSQLException( "Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable); } if (!incrementalSQL.contains("`_hoodie_commit_time` > '%targetBasePath'")) { log.info("Incremental SQL : " + incrementalSQL - + " does not contain `_hoodie_commit_time` > '%targetBasePath'. Please add this clause for incremental to work properly."); + + " does not contain `_hoodie_commit_time` > '%targetBasePath'. Please add " + + "this clause for incremental to work properly."); throw new HoodieIncrementalPullSQLException( - "Incremental SQL does not have clause `_hoodie_commit_time` > '%targetBasePath', which means its not pulling incrementally"); + "Incremental SQL does not have clause `_hoodie_commit_time` > '%targetBasePath', which " + + "means its not pulling incrementally"); } incrementalPullSQLtemplate @@ -227,11 +229,12 @@ public class HiveIncrementalPuller { // Set the hoodie modie executeStatement("set hoodie." + config.sourceTable + ".consume.mode=INCREMENTAL", stmt); // Set the from commit time - executeStatement("set hoodie." + config.sourceTable + ".consume.start.timestamp=" - + config.fromCommitTime, stmt); + executeStatement( + "set hoodie." + config.sourceTable + ".consume.start.timestamp=" + config.fromCommitTime, + stmt); // Set number of commits to pull - executeStatement("set hoodie." + config.sourceTable + ".consume.max.commits=" + String - .valueOf(config.maxCommits), stmt); + executeStatement("set hoodie." + config.sourceTable + ".consume.max.commits=" + String.valueOf( + config.maxCommits), stmt); } private boolean deleteHDFSPath(FileSystem fs, String path) throws IOException { @@ -245,8 +248,9 @@ public class HiveIncrementalPuller { } private String inferCommitTime(FileSystem fs) throws SQLException, IOException { - log.info("FromCommitTime not specified. Trying to infer it from Hoodie dataset " - + config.targetDb + "." + config.targetTable); + log.info( + "FromCommitTime not specified. Trying to infer it from Hoodie dataset " + config.targetDb + + "." + config.targetTable); String targetDataLocation = getTableLocation(config.targetDb, config.targetTable); return scanForCommitTime(fs, targetDataLocation); } @@ -260,8 +264,8 @@ public class HiveIncrementalPuller { resultSet = stmt.executeQuery("describe formatted `" + db + "." + table + "`"); while (resultSet.next()) { if (resultSet.getString(1).trim().equals("Location:")) { - log.info("Inferred table location for " + db + "." + table + " as " + resultSet - .getString(2)); + log.info( + "Inferred table location for " + db + "." + table + " as " + resultSet.getString(2)); return resultSet.getString(2); } } @@ -293,8 +297,7 @@ public class HiveIncrementalPuller { } HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), targetDataPath); - Optional - lastCommit = metadata.getActiveTimeline().getCommitsTimeline() + Optional lastCommit = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants().lastInstant(); if (lastCommit.isPresent()) { return lastCommit.get().getTimestamp(); @@ -302,8 +305,7 @@ public class HiveIncrementalPuller { return "0"; } - private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) - throws IOException { + private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) throws IOException { Path targetBaseDirPath = new Path(config.hoodieTmpDir, config.targetTable + "__" + config.sourceTable); if (!fs.exists(targetBaseDirPath)) { @@ -320,8 +322,7 @@ public class HiveIncrementalPuller { if (fs.exists(targetPath)) { boolean result = fs.delete(targetPath, true); if (!result) { - throw new HoodieException( - "Could not delete existing " + targetPath); + throw new HoodieException("Could not delete existing " + targetPath); } } log.info("Creating " + targetPath + " with permission drwxrwxrwx"); @@ -334,15 +335,14 @@ public class HiveIncrementalPuller { HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), sourceTableLocation); List commitsToSync = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants() - .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants() - .map(HoodieInstant::getTimestamp) + .findInstantsAfter(config.fromCommitTime, config.maxCommits) + .getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); if (commitsToSync.isEmpty()) { - log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + metadata - .getActiveTimeline().getCommitsTimeline().filterCompletedInstants() - .getInstants() - .collect(Collectors.toList()) + " and from commit time is " - + config.fromCommitTime); + log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + + metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() + .getInstants().collect(Collectors.toList()) + + " and from commit time is " + config.fromCommitTime); return null; } log.info("Syncing commits " + commitsToSync); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java index 41ef77276..a91e8f396 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java @@ -77,19 +77,18 @@ public class HoodieSnapshotCopier implements Serializable { final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir); final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView( tableMetadata, - tableMetadata.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants()); + tableMetadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()); // Get the latest commit - Optional latestCommit = tableMetadata.getActiveTimeline() - .getCommitsTimeline().filterCompletedInstants().lastInstant(); + Optional latestCommit = tableMetadata.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().lastInstant(); if (!latestCommit.isPresent()) { logger.warn("No commits present. Nothing to snapshot"); return; } final String latestCommitTimestamp = latestCommit.get().getTimestamp(); - logger.info(String - .format("Starting to snapshot latest version files which are also no-late-than %s.", - latestCommitTimestamp)); + logger.info(String.format( + "Starting to snapshot latest version files which are also no-late-than %s.", + latestCommitTimestamp)); List partitions = FSUtils .getAllPartitionPaths(fs, baseDir, shouldAssumeDatePartitioning); @@ -104,25 +103,24 @@ public class HoodieSnapshotCopier implements Serializable { fs.delete(new Path(outputDir), true); } - jsc.parallelize(partitions, partitions.size()) - .flatMap(partition -> { - // Only take latest version files <= latestCommit. - FileSystem fs1 = FSUtils.getFs(baseDir, serConf.get()); - List> filePaths = new ArrayList<>(); - Stream dataFiles = fsView - .getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp); - dataFiles.forEach( - hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()))); + jsc.parallelize(partitions, partitions.size()).flatMap(partition -> { + // Only take latest version files <= latestCommit. + FileSystem fs1 = FSUtils.getFs(baseDir, serConf.get()); + List> filePaths = new ArrayList<>(); + Stream dataFiles = fsView.getLatestDataFilesBeforeOrOn(partition, + latestCommitTimestamp); + dataFiles.forEach( + hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()))); - // also need to copy over partition metadata - Path partitionMetaFile = new Path(new Path(baseDir, partition), - HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE); - if (fs1.exists(partitionMetaFile)) { - filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString())); - } + // also need to copy over partition metadata + Path partitionMetaFile = new Path(new Path(baseDir, partition), + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE); + if (fs1.exists(partitionMetaFile)) { + filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString())); + } - return filePaths.iterator(); - }).foreach(tuple -> { + return filePaths.iterator(); + }).foreach(tuple -> { String partition = tuple._1(); Path sourceFilePath = new Path(tuple._2()); Path toPartitionPath = new Path(outputDir, partition); @@ -131,8 +129,8 @@ public class HoodieSnapshotCopier implements Serializable { if (!ifs.exists(toPartitionPath)) { ifs.mkdirs(toPartitionPath); } - FileUtil.copy(ifs, sourceFilePath, ifs, - new Path(toPartitionPath, sourceFilePath.getName()), false, ifs.getConf()); + FileUtil.copy(ifs, sourceFilePath, ifs, new Path(toPartitionPath, sourceFilePath.getName()), + false, ifs.getConf()); }); // Also copy the .commit files @@ -143,23 +141,21 @@ public class HoodieSnapshotCopier implements Serializable { if (commitFilePath.getName().equals(HoodieTableConfig.HOODIE_PROPERTIES_FILE)) { return true; } else { - String commitTime = - FSUtils.getCommitFromCommitFile(commitFilePath.getName()); + String commitTime = FSUtils.getCommitFromCommitFile(commitFilePath.getName()); return HoodieTimeline.compareTimestamps(commitTime, latestCommitTimestamp, HoodieTimeline.LESSER_OR_EQUAL); } }); for (FileStatus commitStatus : commitFilesToCopy) { Path targetFilePath = new Path( - outputDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitStatus - .getPath().getName()); + outputDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitStatus.getPath() + .getName()); if (!fs.exists(targetFilePath.getParent())) { fs.mkdirs(targetFilePath.getParent()); } if (fs.exists(targetFilePath)) { - logger.error(String - .format("The target output commit file (%targetBasePath) already exists.", - targetFilePath)); + logger.error(String.format( + "The target output commit file (%targetBasePath) already exists.", targetFilePath)); } FileUtil.copy(fs, commitStatus.getPath(), fs, targetFilePath, false, fs.getConf()); } @@ -179,9 +175,8 @@ public class HoodieSnapshotCopier implements Serializable { // Take input configs final Config cfg = new Config(); new JCommander(cfg, args); - logger.info(String - .format("Snapshot hoodie table from %targetBasePath to %targetBasePath", cfg.basePath, - cfg.outputPath)); + logger.info(String.format("Snapshot hoodie table from %targetBasePath to %targetBasePath", + cfg.basePath, cfg.outputPath)); // Create a spark job to do the snapshot copy SparkConf sparkConf = new SparkConf().setAppName("Hoodie-snapshot-copier"); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java index 502e36e83..d64829ea6 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java @@ -41,9 +41,8 @@ public class UtilHelpers { JavaSparkContext jssc, SourceDataFormat dataFormat, SchemaProvider schemaProvider) throws IOException { try { - return (Source) ConstructorUtils - .invokeConstructor(Class.forName(sourceClass), (Object) cfg, (Object) jssc, - (Object) dataFormat, (Object) schemaProvider); + return (Source) ConstructorUtils.invokeConstructor(Class.forName(sourceClass), (Object) cfg, + (Object) jssc, (Object) dataFormat, (Object) schemaProvider); } catch (Throwable e) { throw new IOException("Could not load source class " + sourceClass, e); } @@ -52,8 +51,8 @@ public class UtilHelpers { public static SchemaProvider createSchemaProvider(String schemaProviderClass, PropertiesConfiguration cfg) throws IOException { try { - return (SchemaProvider) ConstructorUtils - .invokeConstructor(Class.forName(schemaProviderClass), (Object) cfg); + return (SchemaProvider) ConstructorUtils.invokeConstructor(Class.forName(schemaProviderClass), + (Object) cfg); } catch (Throwable e) { throw new IOException("Could not load schema provider class " + schemaProviderClass, e); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index eadb5cd49..f4619465c 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -120,9 +120,8 @@ public class HoodieDeltaStreamer implements Serializable { if (fs.exists(new Path(cfg.targetBasePath))) { HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), cfg.targetBasePath); - this.commitTimelineOpt = Optional - .of(meta.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants()); + this.commitTimelineOpt = Optional.of(meta.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants()); } else { this.commitTimelineOpt = Optional.empty(); } @@ -137,13 +136,13 @@ public class HoodieDeltaStreamer implements Serializable { // Create the source & schema providers PropertiesConfiguration sourceCfg = UtilHelpers.readConfig(fs, new Path(cfg.sourceConfigProps)); log.info("Creating source " + cfg.sourceClassName + " with configs : " + sourceCfg.toString()); - this.source = UtilHelpers - .createSource(cfg.sourceClassName, sourceCfg, jssc, cfg.sourceFormat, schemaProvider); + this.source = UtilHelpers.createSource(cfg.sourceClassName, sourceCfg, jssc, cfg.sourceFormat, + schemaProvider); } private void initSchemaProvider() throws IOException { - PropertiesConfiguration schemaCfg = UtilHelpers - .readConfig(fs, new Path(cfg.schemaProviderConfigProps)); + PropertiesConfiguration schemaCfg = UtilHelpers.readConfig(fs, + new Path(cfg.schemaProviderConfigProps)); log.info( "Creating schema provider " + cfg.schemaProviderClassName + " with configs : " + schemaCfg .toString()); @@ -175,8 +174,8 @@ public class HoodieDeltaStreamer implements Serializable { sparkConf = HoodieWriteClient.registerClasses(sparkConf); // register the schemas, so that shuffle does not serialize the full schemas - List schemas = Arrays - .asList(schemaProvider.getSourceSchema(), schemaProvider.getTargetSchema()); + List schemas = Arrays.asList(schemaProvider.getSourceSchema(), + schemaProvider.getTargetSchema()); sparkConf.registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList()); return new JavaSparkContext(sparkConf); } @@ -187,15 +186,14 @@ public class HoodieDeltaStreamer implements Serializable { if (commitTimelineOpt.isPresent()) { Optional lastCommit = commitTimelineOpt.get().lastInstant(); if (lastCommit.isPresent()) { - HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata - .fromBytes(commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get()); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get()); if (commitMetadata.getMetadata(CHECKPOINT_KEY) != null) { resumeCheckpointStr = Optional.of(commitMetadata.getMetadata(CHECKPOINT_KEY)); } else { throw new HoodieDeltaStreamerException( - "Unable to find previous checkpoint. Please double check if this table " + - "was indeed built via delta streamer "); + "Unable to find previous checkpoint. Please double check if this table " + + "was indeed built via delta streamer "); } } } else { @@ -208,8 +206,8 @@ public class HoodieDeltaStreamer implements Serializable { log.info("Checkpoint to resume from : " + resumeCheckpointStr); // Pull the data from the source & prepare the write - Pair>, String> dataAndCheckpoint = source - .fetchNewData(resumeCheckpointStr, cfg.maxInputBytes); + Pair>, String> dataAndCheckpoint = source.fetchNewData( + resumeCheckpointStr, cfg.maxInputBytes); if (!dataAndCheckpoint.getKey().isPresent()) { log.info("No new data, nothing to commit.. "); @@ -217,14 +215,11 @@ public class HoodieDeltaStreamer implements Serializable { } JavaRDD avroRDD = dataAndCheckpoint.getKey().get(); - JavaRDD records = avroRDD - .map(gr -> { - HoodieRecordPayload payload = DataSourceUtils.createPayload( - cfg.payloadClassName, - gr, - (Comparable) gr.get(cfg.sourceOrderingField)); - return new HoodieRecord<>(keyGenerator.getKey(gr), payload); - }); + JavaRDD records = avroRDD.map(gr -> { + HoodieRecordPayload payload = DataSourceUtils.createPayload(cfg.payloadClassName, gr, + (Comparable) gr.get(cfg.sourceOrderingField)); + return new HoodieRecord<>(keyGenerator.getKey(gr), payload); + }); // Perform the write HoodieWriteConfig hoodieCfg = getHoodieClientConfig(cfg.hoodieClientProps); @@ -245,8 +240,8 @@ public class HoodieDeltaStreamer implements Serializable { HashMap checkpointCommitMetadata = new HashMap<>(); checkpointCommitMetadata.put(CHECKPOINT_KEY, dataAndCheckpoint.getValue()); - boolean success = client - .commit(commitTime, writeStatusRDD, Optional.of(checkpointCommitMetadata)); + boolean success = client.commit(commitTime, writeStatusRDD, + Optional.of(checkpointCommitMetadata)); if (success) { log.info("Commit " + commitTime + " successful!"); // TODO(vc): Kick off hive sync from here. @@ -258,23 +253,20 @@ public class HoodieDeltaStreamer implements Serializable { } private HoodieWriteConfig getHoodieClientConfig(String hoodieClientCfgPath) throws Exception { - return HoodieWriteConfig.newBuilder() - .combineInput(true, true) - .withPath(cfg.targetBasePath) - .withAutoCommit(false) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withPayloadClass(OverwriteWithLatestAvroPayload.class.getName()).build()) + return HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath) + .withAutoCommit(false).withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withPayloadClass( + OverwriteWithLatestAvroPayload + .class + .getName()).build()) .withSchema(schemaProvider.getTargetSchema().toString()) - .forTable(cfg.targetTableName) - .withIndexConfig( + .forTable(cfg.targetTableName).withIndexConfig( HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .fromInputStream(fs.open(new Path(hoodieClientCfgPath))) - .build(); + .fromInputStream(fs.open(new Path(hoodieClientCfgPath))).build(); } private enum Operation { - UPSERT, - INSERT + UPSERT, INSERT } private class OperationConvertor implements IStringConverter { @@ -308,58 +300,86 @@ public class HoodieDeltaStreamer implements Serializable { public String targetTableName; @Parameter(names = {"--hoodie-client-config"}, description = - "path to properties file on localfs or dfs, with hoodie client config. Sane defaults" + - "are used, but recommend use to provide basic things like metrics endpoints, hive configs etc") + "path to properties file on localfs or " + + "dfs, with hoodie client config. " + + "Sane defaults" + + "are used, but recommend use to " + + "provide basic things like metrics " + + "endpoints, hive configs etc") public String hoodieClientProps = null; /** * SOURCE CONFIGS **/ @Parameter(names = {"--source-class"}, description = - "subclass of com.uber.hoodie.utilities.sources.Source to use to read data. " + - "built-in options: com.uber.hoodie.utilities.common.{DFSSource (default), KafkaSource, HiveIncrPullSource}") + "subclass of com.uber.hoodie.utilities.sources" + + ".Source to use to read data. " + + "built-in options: com.uber.hoodie.utilities" + + ".common.{DFSSource (default), KafkaSource, " + + "HiveIncrPullSource}") public String sourceClassName = DFSSource.class.getName(); @Parameter(names = {"--source-config"}, description = - "path to properties file on localfs or dfs, with source configs. " + - "For list of acceptable properties, refer the source class", required = true) + "path to properties file on localfs or dfs, with " + + "source configs. " + + "For list of acceptable properties, refer " + + "the source class", required = true) public String sourceConfigProps = null; @Parameter(names = {"--source-format"}, description = - "Format of data in source, JSON (default), Avro. All source data is " + - "converted to Avro using the provided schema in any case", converter = SourceFormatConvertor.class) + "Format of data in source, JSON (default), Avro. " + + "All source data is " + + "converted to Avro using the provided " + + "schema in any case", converter = SourceFormatConvertor.class) public SourceDataFormat sourceFormat = SourceDataFormat.JSON; @Parameter(names = {"--source-ordering-field"}, description = - "Field within source record to decide how to break ties between " + - " records with same key in input data. Default: 'ts' holding unix timestamp of record") + "Field within source record to decide how" + + " to break ties between " + + " records with same key in input " + + "data. Default: 'ts' holding unix " + + "timestamp of record") public String sourceOrderingField = "ts"; @Parameter(names = {"--key-generator-class"}, description = - "Subclass of com.uber.hoodie.utilities.common.KeyExtractor to generate" + - "a HoodieKey from the given avro record. Built in: SimpleKeyGenerator (Uses provided field names as recordkey & partitionpath. " - + - "Nested fields specified via dot notation, e.g: a.b.c)") + "Subclass of com.uber.hoodie.utilities" + + ".common.KeyExtractor to generate" + + "a HoodieKey from the given avro " + + "record. Built in: SimpleKeyGenerator" + + " (Uses provided field names as " + + "recordkey & partitionpath. " + + "Nested fields specified via dot " + + "notation, e.g: a.b.c)") public String keyGeneratorClass = SimpleKeyGenerator.class.getName(); @Parameter(names = {"--key-generator-config"}, description = - "Path to properties file on localfs or dfs, with KeyGenerator configs. " + - "For list of acceptable properites, refer the KeyGenerator class", required = true) + "Path to properties file on localfs or " + + "dfs, with KeyGenerator configs. " + + "For list of acceptable properites, " + + "refer the KeyGenerator class", + required = true) public String keyGeneratorProps = null; @Parameter(names = {"--payload-class"}, description = - "subclass of HoodieRecordPayload, that works off a GenericRecord. " + - "Default: SourceWrapperPayload. Implement your own, if you want to do something other than overwriting existing value") + "subclass of HoodieRecordPayload, that works off " + + "a GenericRecord. " + + "Default: SourceWrapperPayload. Implement " + + "your own, if you want to do something " + + "other than overwriting existing value") public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName(); @Parameter(names = {"--schemaprovider-class"}, description = - "subclass of com.uber.hoodie.utilities.schema.SchemaProvider " + - "to attach schemas to input & target table data, built in options: FilebasedSchemaProvider") + "subclass of com.uber.hoodie.utilities" + + ".schema.SchemaProvider " + + "to attach schemas to input & target" + + " table data, built in options: " + + "FilebasedSchemaProvider") public String schemaProviderClassName = FilebasedSchemaProvider.class.getName(); @Parameter(names = {"--schemaprovider-config"}, description = - "path to properties file on localfs or dfs, with schema configs. " + - "For list of acceptable properties, refer the schema provider class", required = true) + "path to properties file on localfs or dfs, with schema " + + "configs. For list of acceptable properties, refer " + + "the schema provider class", required = true) public String schemaProviderConfigProps = null; @@ -371,8 +391,9 @@ public class HoodieDeltaStreamer implements Serializable { public long maxInputBytes = 1L * 1024 * 1024 * 1024 * 1024; @Parameter(names = {"--op"}, description = - "Takes one of these values : UPSERT (default), INSERT (use when input " + - "is purely new data/inserts to gain speed)", converter = OperationConvertor.class) + "Takes one of these values : UPSERT (default), INSERT (use when input " + + "is purely new data/inserts to gain speed)", + converter = OperationConvertor.class) public Operation operation = Operation.UPSERT; diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java index d9da949b5..a0c8e4a62 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/keygen/TimestampBasedKeyGenerator.java @@ -38,9 +38,7 @@ import org.apache.commons.configuration.PropertiesConfiguration; public class TimestampBasedKeyGenerator extends SimpleKeyGenerator { enum TimestampType implements Serializable { - UNIX_TIMESTAMP, - DATE_STRING, - MIXED + UNIX_TIMESTAMP, DATE_STRING, MIXED } private final TimestampType timestampType; @@ -56,9 +54,14 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator { static class Config { // One value from TimestampType above - private static final String TIMESTAMP_TYPE_FIELD_PROP = "hoodie.deltastreamer.keygen.timebased.timestamp.type"; - private static final String TIMESTAMP_INPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformat"; - private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.dateformat"; + private static final String TIMESTAMP_TYPE_FIELD_PROP = "hoodie.deltastreamer.keygen" + + ".timebased.timestamp.type"; + private static final String TIMESTAMP_INPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen" + + ".timebased.input" + + ".dateformat"; + private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen" + + ".timebased.output" + + ".dateformat"; } public TimestampBasedKeyGenerator(PropertiesConfiguration config) { diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java index 44745b093..aa38bf8b9 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java @@ -39,8 +39,12 @@ public class FilebasedSchemaProvider extends SchemaProvider { */ static class Config { - private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased.schemaprovider.source.schema.file"; - private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased.schemaprovider.target.schema.file"; + private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased" + + ".schemaprovider.source.schema" + + ".file"; + private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.filebased" + + ".schemaprovider.target.schema" + + ".file"; } private final FileSystem fs; @@ -56,10 +60,10 @@ public class FilebasedSchemaProvider extends SchemaProvider { DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP)); try { - this.sourceSchema = new Schema.Parser() - .parse(fs.open(new Path(config.getString(Config.SOURCE_SCHEMA_FILE_PROP)))); - this.targetSchema = new Schema.Parser() - .parse(fs.open(new Path(config.getString(Config.TARGET_SCHEMA_FILE_PROP)))); + this.sourceSchema = new Schema.Parser().parse( + fs.open(new Path(config.getString(Config.SOURCE_SCHEMA_FILE_PROP)))); + this.targetSchema = new Schema.Parser().parse( + fs.open(new Path(config.getString(Config.TARGET_SCHEMA_FILE_PROP)))); } catch (IOException ioe) { throw new HoodieIOException("Error reading schema", ioe); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java index f9c9a9a1d..a6bd623f6 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java @@ -55,10 +55,10 @@ public class DFSSource extends Source { */ static class Config { - private final static String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root"; + private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root"; } - private final static List IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_"); + private static final List IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_"); private final transient FileSystem fs; @@ -73,9 +73,7 @@ public class DFSSource extends Source { public static JavaRDD fromAvroFiles(final AvroConvertor convertor, String pathStr, JavaSparkContext sparkContext) { JavaPairRDD avroRDD = sparkContext.newAPIHadoopFile(pathStr, - AvroKeyInputFormat.class, - AvroKey.class, - NullWritable.class, + AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, sparkContext.hadoopConfiguration()); return avroRDD.keys().map(r -> ((GenericRecord) r.datum())); } @@ -106,28 +104,28 @@ public class DFSSource extends Source { try { // obtain all eligible files under root folder. List eligibleFiles = new ArrayList<>(); - RemoteIterator fitr = fs - .listFiles(new Path(config.getString(Config.ROOT_INPUT_PATH_PROP)), true); + RemoteIterator fitr = fs.listFiles( + new Path(config.getString(Config.ROOT_INPUT_PATH_PROP)), true); while (fitr.hasNext()) { LocatedFileStatus fileStatus = fitr.next(); - if (fileStatus.isDirectory() || - IGNORE_FILEPREFIX_LIST.stream() - .filter(pfx -> fileStatus.getPath().getName().startsWith(pfx)).count() > 0) { + if (fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream().filter( + pfx -> fileStatus.getPath().getName().startsWith(pfx)).count() > 0) { continue; } eligibleFiles.add(fileStatus); } // sort them by modification time. eligibleFiles.sort((FileStatus f1, FileStatus f2) -> Long.valueOf(f1.getModificationTime()) - .compareTo(Long.valueOf(f2.getModificationTime()))); + .compareTo(Long.valueOf( + f2.getModificationTime()))); // Filter based on checkpoint & input size, if needed long currentBytes = 0; long maxModificationTime = Long.MIN_VALUE; List filteredFiles = new ArrayList<>(); for (FileStatus f : eligibleFiles) { - if (lastCheckpointStr.isPresent() && f.getModificationTime() <= Long - .valueOf(lastCheckpointStr.get())) { + if (lastCheckpointStr.isPresent() && f.getModificationTime() <= Long.valueOf( + lastCheckpointStr.get())) { // skip processed files continue; } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java index aeecb9db0..2b4de59ee 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java @@ -44,12 +44,12 @@ import org.apache.spark.api.java.JavaSparkContext; /** * Source to read deltas produced by {@link com.uber.hoodie.utilities.HiveIncrementalPuller}, commit * by commit and apply to the target table - * + *

* The general idea here is to have commits sync across the data pipeline. - * + *

* [Source Tables(s)] ====> HiveIncrementalScanner ==> incrPullRootPath ==> targetTable * {c1,c2,c3,...} {c1,c2,c3,...} {c1,c2,c3,...} - * + *

* This produces beautiful causality, that makes data issues in ETLs very easy to debug */ public class HiveIncrPullSource extends Source { @@ -66,7 +66,7 @@ public class HiveIncrPullSource extends Source { */ static class Config { - private final static String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.incrpull.root"; + private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.incrpull.root"; } public HiveIncrPullSource(PropertiesConfiguration config, JavaSparkContext sparkContext, @@ -121,8 +121,8 @@ public class HiveIncrPullSource extends Source { } // read the files out. - List commitDeltaFiles = Arrays - .asList(fs.listStatus(new Path(incrPullRootPath, commitToPull.get()))); + List commitDeltaFiles = Arrays.asList( + fs.listStatus(new Path(incrPullRootPath, commitToPull.get()))); String pathStr = commitDeltaFiles.stream().map(f -> f.getPath().toString()) .collect(Collectors.joining(",")); String schemaStr = schemaProvider.getSourceSchema().toString(); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java index 36c43c596..f470848f1 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java @@ -84,7 +84,7 @@ public class KafkaSource extends Source { /** * String representation of checkpoint - * + *

* Format: topic1,0:offset0,1:offset1,2:offset2, ..... */ public static String offsetsToStr( @@ -132,9 +132,7 @@ public class KafkaSource extends Source { static class ScalaHelpers { public static Map toScalaMap(HashMap m) { - return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap( - Predef.conforms() - ); + return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap(Predef.conforms()); } public static Set toScalaSet(HashSet s) { @@ -152,8 +150,8 @@ public class KafkaSource extends Source { */ static class Config { - private final static String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic"; - private final static String DEFAULT_AUTO_RESET_OFFSET = "largest"; + private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic"; + private static final String DEFAULT_AUTO_RESET_OFFSET = "largest"; } @@ -166,8 +164,8 @@ public class KafkaSource extends Source { super(config, sparkContext, dataFormat, schemaProvider); kafkaParams = new HashMap<>(); - Stream keys = StreamSupport - .stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.NONNULL), false); + Stream keys = StreamSupport.stream( + Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.NONNULL), false); keys.forEach(k -> kafkaParams.put(k, config.getString(k))); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.KAFKA_TOPIC_NAME)); @@ -180,8 +178,8 @@ public class KafkaSource extends Source { // Obtain current metadata for the topic KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams)); - Either, Set> either = cluster - .getPartitions(ScalaHelpers.toScalaSet(new HashSet<>(Arrays.asList(topicName)))); + Either, Set> either = cluster.getPartitions( + ScalaHelpers.toScalaSet(new HashSet<>(Arrays.asList(topicName)))); if (either.isLeft()) { // log errors. and bail out. throw new HoodieDeltaStreamerException("Error obtaining partition metadata", @@ -197,8 +195,8 @@ public class KafkaSource extends Source { String autoResetValue = config .getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET); if (autoResetValue.equals("smallest")) { - fromOffsets = new HashMap(ScalaHelpers - .toJavaMap(cluster.getEarliestLeaderOffsets(topicPartitions).right().get())); + fromOffsets = new HashMap(ScalaHelpers.toJavaMap( + cluster.getEarliestLeaderOffsets(topicPartitions).right().get())); } else if (autoResetValue.equals("largest")) { fromOffsets = new HashMap( ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get())); @@ -213,7 +211,8 @@ public class KafkaSource extends Source { ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get())); // Come up with final set of OffsetRanges to read (account for new partitions) - // TODO(vc): Respect maxInputBytes, by estimating number of messages to read each batch from partition size + // TODO(vc): Respect maxInputBytes, by estimating number of messages to read each batch from + // partition size OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets); long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges); if (totalNewMsgs <= 0) { @@ -225,14 +224,8 @@ public class KafkaSource extends Source { } // Perform the actual read from Kafka - JavaRDD kafkaRDD = KafkaUtils.createRDD( - sparkContext, - byte[].class, - byte[].class, - DefaultDecoder.class, - DefaultDecoder.class, - kafkaParams, - offsetRanges).values(); + JavaRDD kafkaRDD = KafkaUtils.createRDD(sparkContext, byte[].class, byte[].class, + DefaultDecoder.class, DefaultDecoder.class, kafkaParams, offsetRanges).values(); // Produce a RDD[GenericRecord] final AvroConvertor avroConvertor = new AvroConvertor( @@ -241,8 +234,8 @@ public class KafkaSource extends Source { if (dataFormat == SourceDataFormat.AVRO) { newDataRDD = kafkaRDD.map(bytes -> avroConvertor.fromAvroBinary(bytes)); } else if (dataFormat == SourceDataFormat.JSON) { - newDataRDD = kafkaRDD - .map(bytes -> avroConvertor.fromJson(new String(bytes, Charset.forName("utf-8")))); + newDataRDD = kafkaRDD.map( + bytes -> avroConvertor.fromJson(new String(bytes, Charset.forName("utf-8")))); } else { throw new HoodieNotSupportedException("Unsupport data format :" + dataFormat); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java index d8ff58e89..fe7a805d4 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java @@ -54,8 +54,7 @@ public abstract class Source implements Serializable { * data, as well as the checkpoint to be written as a result of that. */ public abstract Pair>, String> fetchNewData( - Optional lastCheckpointStr, - long maxInputBytes); + Optional lastCheckpointStr, long maxInputBytes); public PropertiesConfiguration getConfig() { diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java index c1e58153f..65a5d972c 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java @@ -16,7 +16,6 @@ package com.uber.hoodie.utilities; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -103,9 +102,8 @@ public class TestHDFSParquetImporter implements Serializable { createRecords(srcFolder); HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(), - hoodieFolder.toString(), - "testTable", "COPY_ON_WRITE", "_row_key", "timestamp", - 1, schemaFile); + hoodieFolder.toString(), "testTable", "COPY_ON_WRITE", "_row_key", "timestamp", 1, + schemaFile); AtomicInteger retry = new AtomicInteger(3); AtomicInteger fileCreated = new AtomicInteger(0); HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg) { @@ -168,8 +166,7 @@ public class TestHDFSParquetImporter implements Serializable { .generateGenericRecord(Long.toString(recordNum), "rider-" + recordNum, "driver-" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum))); } - ParquetWriter writer = AvroParquetWriter - .builder(srcFile) + ParquetWriter writer = AvroParquetWriter.builder(srcFile) .withSchema(HoodieTestDataGenerator.avroSchema) .withConf(HoodieTestUtils.getDefaultHadoopConf()) .build(); @@ -202,9 +199,8 @@ public class TestHDFSParquetImporter implements Serializable { Path srcFolder = new Path(basePath.toString(), "srcTest"); Path schemaFile = new Path(basePath.toString(), "missingFile.schema"); HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(), - hoodieFolder.toString(), - "testTable", "COPY_ON_WRITE", "_row_key", "timestamp", - 1, schemaFile.toString()); + hoodieFolder.toString(), "testTable", "COPY_ON_WRITE", "_row_key", "timestamp", 1, + schemaFile.toString()); HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg); // Should fail - return : -1. assertEquals(-1, dataImporter.dataImport(jsc, 0)); @@ -247,16 +243,14 @@ public class TestHDFSParquetImporter implements Serializable { HDFSParquetImporter.Config cfg; // Check for invalid row key. - cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), - "testTable", "COPY_ON_WRITE", "invalidRowKey", "timestamp", - 1, schemaFile.toString()); + cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), "testTable", + "COPY_ON_WRITE", "invalidRowKey", "timestamp", 1, schemaFile.toString()); dataImporter = new HDFSParquetImporter(cfg); assertEquals(-1, dataImporter.dataImport(jsc, 0)); // Check for invalid partition key. - cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), - "testTable", "COPY_ON_WRITE", "_row_key", "invalidTimeStamp", - 1, schemaFile.toString()); + cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(), "testTable", + "COPY_ON_WRITE", "_row_key", "invalidTimeStamp", 1, schemaFile.toString()); dataImporter = new HDFSParquetImporter(cfg); assertEquals(-1, dataImporter.dataImport(jsc, 0)); diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java index f52233339..45344da0d 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java @@ -97,9 +97,9 @@ public class TestHoodieSnapshotCopier { new File(basePath + "/2016/05/01/").mkdirs(); new File(basePath + "/2016/05/02/").mkdirs(); new File(basePath + "/2016/05/06/").mkdirs(); - HoodieTestDataGenerator.writePartitionMetadata(fs, - new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, - basePath); + HoodieTestDataGenerator + .writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"}, + basePath); // Make commit1 File file11 = new File( basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, 1, "id11")); diff --git a/pom.xml b/pom.xml index 7ed46636b..f8240c482 100644 --- a/pom.xml +++ b/pom.xml @@ -151,6 +151,40 @@ + + org.apache.maven.plugins + maven-checkstyle-plugin + 3.0.0 + + + com.puppycrawl.tools + checkstyle + 8.8 + + + + true + UTF-8 + style/checkstyle.xml + style/checkstyle-suppressions.xml + checkstyle.suppressions.file + true + warning + true + + ${project.build.sourceDirectory} + + **\/generated-sources\/ + + + + compile + + check + + + + org.apache.maven.plugins maven-compiler-plugin @@ -287,6 +321,7 @@ **/test/resources/*.csv **/main/avro/*.avsc **/target/* + **/style/* diff --git a/style/checkstyle-suppressions.xml b/style/checkstyle-suppressions.xml new file mode 100644 index 000000000..02a7a4a0f --- /dev/null +++ b/style/checkstyle-suppressions.xml @@ -0,0 +1,12 @@ + + + + + + + + + + diff --git a/style/checkstyle.xml b/style/checkstyle.xml new file mode 100644 index 000000000..c0667b274 --- /dev/null +++ b/style/checkstyle.xml @@ -0,0 +1,246 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/style/eclipse-java-google-style.xml b/style/eclipse-java-google-style.xml new file mode 100644 index 000000000..d1af2890b --- /dev/null +++ b/style/eclipse-java-google-style.xml @@ -0,0 +1,337 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/style/intellij-java-google-style.xml b/style/intellij-java-google-style.xml new file mode 100644 index 000000000..658087ef6 --- /dev/null +++ b/style/intellij-java-google-style.xml @@ -0,0 +1,598 @@ + + + + + +

+ + + + xmlns:android + + ^$ + + + +
+
+ + + + xmlns:.* + + ^$ + + + BY_NAME + +
+
+ + + + .*:id + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + style + + ^$ + + + +
+
+ + + + .* + + ^$ + + + BY_NAME + +
+
+ + + + .*:.*Style + + http://schemas.android.com/apk/res/android + + + BY_NAME + +
+
+ + + + .*:layout_width + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_height + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_weight + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_margin + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginTop + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginBottom + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginStart + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginEnd + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginLeft + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_marginRight + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:layout_.* + + http://schemas.android.com/apk/res/android + + + BY_NAME + +
+
+ + + + .*:padding + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingTop + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingBottom + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingStart + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingEnd + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingLeft + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .*:paddingRight + + http://schemas.android.com/apk/res/android + + + +
+
+ + + + .* + http://schemas.android.com/apk/res/android + + + BY_NAME + +
+
+ + + + .* + http://schemas.android.com/apk/res-auto + + + BY_NAME + +
+
+ + + + .* + http://schemas.android.com/tools + + + BY_NAME + +
+
+ + + + .* + .* + + + BY_NAME + +
+ + + + + + + + + +