From c2c0f6b13d5b72b3098ed1b343b0a89679f854b3 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Thu, 16 Jan 2020 23:58:47 -0800 Subject: [PATCH] [HUDI-509] Renaming code in sync with cWiki restructuring (#1212) - Storage Type replaced with Table Type (remaining instances) - View types replaced with query types; - ReadOptimized view referred as Snapshot Query - TableFileSystemView sub interfaces renamed to BaseFileOnly and Slice Views - HoodieDataFile renamed to HoodieBaseFile - Hive Sync tool will register RO tables for MOR with a `_ro` suffix - Datasource/Deltastreamer options renamed accordingly - Support fallback to old config values as well, so migration is painless - Config for controlling _ro suffix addition - Renaming DataFile to BaseFile across DTOs, HoodieFileSlice and AbstractTableFileSystemView --- README.md | 8 +- doap_HUDI.rdf | 3 +- docker/demo/hive-batch1.commands | 4 +- .../hive-batch2-after-compaction.commands | 4 +- docker/demo/hive-table-check.commands | 2 +- docker/demo/presto-batch1.commands | 4 +- .../presto-batch2-after-compaction.commands | 4 +- docker/demo/sparksql-batch1.commands | 4 +- docker/demo/sparksql-batch2.commands | 4 +- docker/demo/sparksql-incremental.commands | 10 +- .../cli/commands/FileSystemViewCommand.java | 42 +++--- .../org/apache/hudi/cli/DedupeSparkJob.scala | 6 +- .../apache/hudi/CompactionAdminClient.java | 6 +- .../org/apache/hudi/HoodieReadClient.java | 6 +- .../org/apache/hudi/HoodieWriteClient.java | 10 +- .../hudi/index/bloom/HoodieBloomIndex.java | 4 +- .../apache/hudi/io/HoodieAppendHandle.java | 4 +- .../org/apache/hudi/io/HoodieCleanHelper.java | 12 +- .../apache/hudi/io/HoodieKeyLookupHandle.java | 4 +- .../org/apache/hudi/io/HoodieMergeHandle.java | 8 +- .../apache/hudi/io/HoodieRangeInfoHandle.java | 4 +- .../org/apache/hudi/io/HoodieReadHandle.java | 8 +- ...a => HoodieMergeOnReadTableCompactor.java} | 18 +-- .../compact/strategy/CompactionStrategy.java | 8 +- .../LogFileSizeBasedCompactionStrategy.java | 4 +- .../hudi/table/HoodieCopyOnWriteTable.java | 16 +- .../hudi/table/HoodieMergeOnReadTable.java | 29 ++-- .../org/apache/hudi/table/HoodieTable.java | 10 +- .../org/apache/hudi/TestAsyncCompaction.java | 14 +- .../java/org/apache/hudi/TestCleaner.java | 20 +-- .../org/apache/hudi/TestClientRollback.java | 24 +-- .../hudi/TestCompactionAdminClient.java | 4 +- .../org/apache/hudi/TestHoodieClientBase.java | 2 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 18 +-- .../hudi/common/HoodieClientTestUtils.java | 10 +- .../apache/hudi/io/TestHoodieCompactor.java | 4 +- ...eDataFile.java => TestHoodieBaseFile.java} | 10 +- .../TestHoodieCompactionStrategy.java | 4 +- .../hudi/table/TestMergeOnReadTable.java | 110 +++++++------- .../common/model/CompactionOperation.java | 6 +- .../apache/hudi/common/model/FileSlice.java | 18 +-- ...oodieDataFile.java => HoodieBaseFile.java} | 10 +- .../hudi/common/model/HoodieFileGroup.java | 12 +- .../hudi/common/model/HoodieTableType.java | 3 - .../hudi/common/table/HoodieTableConfig.java | 30 ++-- .../common/table/HoodieTableMetaClient.java | 4 +- .../common/table/SyncableFileSystemView.java | 7 +- .../common/table/TableFileSystemView.java | 32 ++-- .../{DataFileDTO.java => BaseFileDTO.java} | 20 +-- .../table/timeline/dto/FileSliceDTO.java | 8 +- .../view/AbstractTableFileSystemView.java | 140 +++++++++--------- ...IncrementalTimelineSyncFileSystemView.java | 10 +- .../view/PriorityBasedFileSystemView.java | 32 ++-- .../view/RemoteHoodieTableFileSystemView.java | 60 ++++---- .../view/RocksDbBasedFileSystemView.java | 24 +-- .../hudi/common/util/CompactionUtils.java | 4 +- .../view/TestHoodieTableFileSystemView.java | 125 ++++++++-------- .../table/view/TestIncrementalFSViewSync.java | 14 +- .../hudi/common/util/CompactionTestUtils.java | 8 +- .../hudi/common/util/TestCompactionUtils.java | 16 +- .../apache/hudi/hadoop/HoodieHiveUtil.java | 11 +- .../hudi/hadoop/HoodieParquetInputFormat.java | 20 +-- .../hudi/hadoop/HoodieROTablePathFilter.java | 6 +- .../AbstractRealtimeRecordReader.java | 4 +- .../realtime/HoodieRealtimeFileSplit.java | 20 +-- .../RealtimeCompactedRecordReader.java | 2 +- .../RealtimeUnmergedRecordReader.java | 2 +- .../org/apache/hudi/hive/HiveSyncConfig.java | 3 + .../org/apache/hudi/hive/HiveSyncTool.java | 86 ++++++----- .../apache/hudi/hive/HoodieHiveClient.java | 101 +++++++------ .../org/apache/hudi/hive/util/SchemaUtil.java | 4 +- .../apache/hudi/hive/TestHiveSyncTool.java | 98 ++++++------ .../java/org/apache/hudi/hive/TestUtil.java | 13 +- .../apache/hudi/integ/ITTestHoodieDemo.java | 19 ++- .../apache/hudi/integ/ITTestHoodieSanity.java | 35 +++-- .../java/org/apache/hudi/DataSourceUtils.java | 4 +- .../org/apache/hudi/DataSourceOptions.scala | 67 ++++++++- .../scala/org/apache/hudi/DefaultSource.scala | 16 +- .../apache/hudi/HoodieSparkSqlWriter.scala | 8 +- hudi-spark/src/test/java/HoodieJavaApp.java | 16 +- .../src/test/java/HoodieJavaStreamingApp.java | 4 +- .../src/test/scala/TestDataSource.scala | 12 +- .../hudi/HoodieSparkSqlWriterSuite.scala | 4 +- .../service/FileSystemViewHandler.java | 22 +-- ...aFileHandler.java => BaseFileHandler.java} | 50 +++---- .../hudi/utilities/HoodieSnapshotCopier.java | 8 +- .../utilities/deltastreamer/DeltaSync.java | 4 +- .../deltastreamer/HoodieDeltaStreamer.java | 17 +-- .../deltastreamer/SchedulerConfGenerator.java | 2 +- .../utilities/sources/HoodieIncrSource.java | 2 +- .../utilities/TestHoodieDeltaStreamer.java | 14 +- .../utilities/TestSchedulerConfGenerator.java | 6 +- 92 files changed, 907 insertions(+), 822 deletions(-) rename hudi-client/src/main/java/org/apache/hudi/io/compact/{HoodieRealtimeTableCompactor.java => HoodieMergeOnReadTableCompactor.java} (94%) rename hudi-client/src/test/java/org/apache/hudi/io/strategy/{TestHoodieDataFile.java => TestHoodieBaseFile.java} (83%) rename hudi-common/src/main/java/org/apache/hudi/common/model/{HoodieDataFile.java => HoodieBaseFile.java} (91%) rename hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/{DataFileDTO.java => BaseFileDTO.java} (77%) rename hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/{DataFileHandler.java => BaseFileHandler.java} (51%) diff --git a/README.md b/README.md index 9284bbb7e..adb630fe1 100644 --- a/README.md +++ b/README.md @@ -35,10 +35,10 @@ Hudi manages the storage of large analytical datasets on DFS (Cloud stores, HDFS * Async compaction of row & columnar data * Timeline metadata to track lineage -Hudi provides the ability to query via three types of views: - * **Read Optimized View** - Provides excellent snapshot query performance via purely columnar storage (e.g. [Parquet](https://parquet.apache.org/)). - * **Incremental View** - Provides a change stream with records inserted or updated after a point in time. - * **Real-time View** - Provides snapshot queries on real-time data, using a combination of columnar & row-based storage (e.g [Parquet](https://parquet.apache.org/) + [Avro](https://avro.apache.org/docs/current/mr.html)). +Hudi supports three types of queries: + * **Snapshot Query** - Provides snapshot queries on real-time data, using a combination of columnar & row-based storage (e.g [Parquet](https://parquet.apache.org/) + [Avro](https://avro.apache.org/docs/current/mr.html)). + * **Incremental Query** - Provides a change stream with records inserted or updated after a point in time. + * **Read Optimized Query** - Provides excellent snapshot query performance via purely columnar storage (e.g. [Parquet](https://parquet.apache.org/)). Learn more about Hudi at [https://hudi.apache.org](https://hudi.apache.org) diff --git a/doap_HUDI.rdf b/doap_HUDI.rdf index 648400412..39db6fa53 100644 --- a/doap_HUDI.rdf +++ b/doap_HUDI.rdf @@ -28,8 +28,7 @@ Ingests and Manages storage of large analytical datasets - Hudi (pronounced “Hoodie”) ingests and manages storage of large analytical datasets over DFS - (HDFS or cloud stores) and provides three logical views for query access. + Hudi (pronounced “Hoodie”) brings stream processing to big data, providing upserts, deletes and incremental data streams. diff --git a/docker/demo/hive-batch1.commands b/docker/demo/hive-batch1.commands index f5e03dd6c..591d7874e 100644 --- a/docker/demo/hive-batch1.commands +++ b/docker/demo/hive-batch1.commands @@ -19,11 +19,11 @@ add jar ${hudi.hadoop.bundle}; select symbol, max(ts) from stock_ticks_cow group by symbol HAVING symbol = 'GOOG'; -select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'; +select symbol, max(ts) from stock_ticks_mor_ro group by symbol HAVING symbol = 'GOOG'; select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'; select symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG'; -select symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'; +select symbol, ts, volume, open, close from stock_ticks_mor_ro where symbol = 'GOOG'; select symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'; !quit diff --git a/docker/demo/hive-batch2-after-compaction.commands b/docker/demo/hive-batch2-after-compaction.commands index 9b6dded59..25cf9eb9c 100644 --- a/docker/demo/hive-batch2-after-compaction.commands +++ b/docker/demo/hive-batch2-after-compaction.commands @@ -18,10 +18,10 @@ add jar ${hudi.hadoop.bundle}; -select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'; +select symbol, max(ts) from stock_ticks_mor_ro group by symbol HAVING symbol = 'GOOG'; select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'; -select symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'; +select symbol, ts, volume, open, close from stock_ticks_mor_ro where symbol = 'GOOG'; select symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'; !quit diff --git a/docker/demo/hive-table-check.commands b/docker/demo/hive-table-check.commands index ed6195b82..9fd4d3c23 100644 --- a/docker/demo/hive-table-check.commands +++ b/docker/demo/hive-table-check.commands @@ -20,7 +20,7 @@ add jar ${hudi.hadoop.bundle}; show tables; show partitions stock_ticks_cow; -show partitions stock_ticks_mor; +show partitions stock_ticks_mor_ro; show partitions stock_ticks_mor_rt; !quit diff --git a/docker/demo/presto-batch1.commands b/docker/demo/presto-batch1.commands index 8b0c960cd..3e39df8d6 100644 --- a/docker/demo/presto-batch1.commands +++ b/docker/demo/presto-batch1.commands @@ -1,4 +1,4 @@ select symbol, max(ts) from stock_ticks_cow group by symbol HAVING symbol = 'GOOG'; -select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'; +select symbol, max(ts) from stock_ticks_mor_ro group by symbol HAVING symbol = 'GOOG'; select symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG'; -select symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'; +select symbol, ts, volume, open, close from stock_ticks_mor_ro where symbol = 'GOOG'; diff --git a/docker/demo/presto-batch2-after-compaction.commands b/docker/demo/presto-batch2-after-compaction.commands index 698d8c0e4..dee46304f 100644 --- a/docker/demo/presto-batch2-after-compaction.commands +++ b/docker/demo/presto-batch2-after-compaction.commands @@ -1,2 +1,2 @@ -select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'; -select symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'; +select symbol, max(ts) from stock_ticks_mor_ro group by symbol HAVING symbol = 'GOOG'; +select symbol, ts, volume, open, close from stock_ticks_mor_ro where symbol = 'GOOG'; diff --git a/docker/demo/sparksql-batch1.commands b/docker/demo/sparksql-batch1.commands index e220cb0bd..727aa1633 100644 --- a/docker/demo/sparksql-batch1.commands +++ b/docker/demo/sparksql-batch1.commands @@ -22,9 +22,9 @@ spark.sql("select symbol, max(ts) from stock_ticks_cow group by symbol HAVING sy spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG'").show(100, false) // Merge-On-Read table -spark.sql("select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'").show(100, false) +spark.sql("select symbol, max(ts) from stock_ticks_mor_ro group by symbol HAVING symbol = 'GOOG'").show(100, false) spark.sql("select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'").show(100, false) -spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'").show(100, false) +spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_ro where symbol = 'GOOG'").show(100, false) spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'").show(100, false) System.exit(0) diff --git a/docker/demo/sparksql-batch2.commands b/docker/demo/sparksql-batch2.commands index 521a4ebe6..391e11b97 100644 --- a/docker/demo/sparksql-batch2.commands +++ b/docker/demo/sparksql-batch2.commands @@ -21,8 +21,8 @@ spark.sql("select symbol, max(ts) from stock_ticks_cow group by symbol HAVING sy spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG'").show(100, false) // Merge-On-Read table -spark.sql("select symbol, max(ts) from stock_ticks_mor group by symbol HAVING symbol = 'GOOG'").show(100, false) -spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor where symbol = 'GOOG'").show(100, false) +spark.sql("select symbol, max(ts) from stock_ticks_mor_ro group by symbol HAVING symbol = 'GOOG'").show(100, false) +spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_ro where symbol = 'GOOG'").show(100, false) spark.sql("select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'").show(100, false) spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'").show(100, false) diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands index ceec945f9..8e3e153e2 100644 --- a/docker/demo/sparksql-incremental.commands +++ b/docker/demo/sparksql-incremental.commands @@ -25,18 +25,18 @@ import org.apache.hadoop.fs.FileSystem; val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) val beginInstantTime = HoodieDataSourceHelpers.listCommitsSince(fs, "/user/hive/warehouse/stock_ticks_cow", "00000").get(0) -val hoodieIncViewDF = spark.read.format("org.apache.hudi"). - option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL). +val hoodieIncQueryDF = spark.read.format("org.apache.hudi"). + option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL). option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, beginInstantTime). load("/user/hive/warehouse/stock_ticks_cow"); -hoodieIncViewDF.registerTempTable("stock_ticks_cow_incr") +hoodieIncQueryDF.registerTempTable("stock_ticks_cow_incr") spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow_incr where symbol = 'GOOG'").show(100, false); spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, close from stock_ticks_cow_incr"). write.format("org.apache.hudi"). option("hoodie.insert.shuffle.parallelism", "2"). option("hoodie.upsert.shuffle.parallelism","2"). - option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_STORAGE_TYPE_OPT_VAL). + option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL). option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL). option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "key"). option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr"). @@ -53,7 +53,7 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl save("/user/hive/warehouse/stock_ticks_derived_mor"); spark.sql("show tables").show(20, false) -spark.sql("select count(*) from stock_ticks_derived_mor").show(20, false) +spark.sql("select count(*) from stock_ticks_derived_mor_ro").show(20, false) spark.sql("select count(*) from stock_ticks_derived_mor_rt").show(20, false) System.exit(0); \ No newline at end of file diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java index 597bab396..93607c37b 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java @@ -60,8 +60,8 @@ public class FileSystemViewCommand implements CommandMarker { public String showAllFileSlices( @CliOption(key = {"pathRegex"}, help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") String globRegex, - @CliOption(key = {"readOptimizedOnly"}, help = "Only display read-optimized view", - unspecifiedDefaultValue = "false") boolean readOptimizedOnly, + @CliOption(key = {"baseFileOnly"}, help = "Only display base files view", + unspecifiedDefaultValue = "false") boolean baseFileOnly, @CliOption(key = {"maxInstant"}, help = "File-Slices upto this instant are displayed", unspecifiedDefaultValue = "") String maxInstant, @CliOption(key = {"includeMax"}, help = "Include Max Instant", @@ -77,19 +77,19 @@ public class FileSystemViewCommand implements CommandMarker { unspecifiedDefaultValue = "false") final boolean headerOnly) throws IOException { - HoodieTableFileSystemView fsView = buildFileSystemView(globRegex, maxInstant, readOptimizedOnly, includeMaxInstant, + HoodieTableFileSystemView fsView = buildFileSystemView(globRegex, maxInstant, baseFileOnly, includeMaxInstant, includeInflight, excludeCompaction); List rows = new ArrayList<>(); fsView.getAllFileGroups().forEach(fg -> fg.getAllFileSlices().forEach(fs -> { int idx = 0; - // For ReadOptimized Views, do not display any delta-file related columns - Comparable[] row = new Comparable[readOptimizedOnly ? 5 : 8]; + // For base file only Views, do not display any delta-file related columns + Comparable[] row = new Comparable[baseFileOnly ? 5 : 8]; row[idx++] = fg.getPartitionPath(); row[idx++] = fg.getFileGroupId().getFileId(); row[idx++] = fs.getBaseInstantTime(); - row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getPath() : ""; - row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getFileSize() : -1; - if (!readOptimizedOnly) { + row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getPath() : ""; + row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getFileSize() : -1; + if (!baseFileOnly) { row[idx++] = fs.getLogFiles().count(); row[idx++] = fs.getLogFiles().mapToLong(HoodieLogFile::getFileSize).sum(); row[idx++] = fs.getLogFiles().collect(Collectors.toList()).toString(); @@ -104,7 +104,7 @@ public class FileSystemViewCommand implements CommandMarker { TableHeader header = new TableHeader().addTableHeaderField("Partition").addTableHeaderField("FileId") .addTableHeaderField("Base-Instant").addTableHeaderField("Data-File").addTableHeaderField("Data-File Size"); - if (!readOptimizedOnly) { + if (!baseFileOnly) { header = header.addTableHeaderField("Num Delta Files").addTableHeaderField("Total Delta File Size") .addTableHeaderField("Delta Files"); } @@ -114,8 +114,8 @@ public class FileSystemViewCommand implements CommandMarker { @CliCommand(value = "show fsview latest", help = "Show latest file-system view") public String showLatestFileSlices( @CliOption(key = {"partitionPath"}, help = "A valid paritition path", mandatory = true) String partition, - @CliOption(key = {"readOptimizedOnly"}, help = "Only display read-optimized view", - unspecifiedDefaultValue = "false") boolean readOptimizedOnly, + @CliOption(key = {"baseFileOnly"}, help = "Only display base file view", + unspecifiedDefaultValue = "false") boolean baseFileOnly, @CliOption(key = {"maxInstant"}, help = "File-Slices upto this instant are displayed", unspecifiedDefaultValue = "") String maxInstant, @CliOption(key = {"merge"}, help = "Merge File Slices due to pending compaction", @@ -133,7 +133,7 @@ public class FileSystemViewCommand implements CommandMarker { unspecifiedDefaultValue = "false") final boolean headerOnly) throws IOException { - HoodieTableFileSystemView fsView = buildFileSystemView(partition, maxInstant, readOptimizedOnly, includeMaxInstant, + HoodieTableFileSystemView fsView = buildFileSystemView(partition, maxInstant, baseFileOnly, includeMaxInstant, includeInflight, excludeCompaction); List rows = new ArrayList<>(); @@ -150,16 +150,16 @@ public class FileSystemViewCommand implements CommandMarker { fileSliceStream.forEach(fs -> { int idx = 0; - Comparable[] row = new Comparable[readOptimizedOnly ? 5 : 13]; + Comparable[] row = new Comparable[baseFileOnly ? 5 : 13]; row[idx++] = partition; row[idx++] = fs.getFileId(); row[idx++] = fs.getBaseInstantTime(); - row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getPath() : ""; + row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getPath() : ""; - long dataFileSize = fs.getDataFile().isPresent() ? fs.getDataFile().get().getFileSize() : -1; + long dataFileSize = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getFileSize() : -1; row[idx++] = dataFileSize; - if (!readOptimizedOnly) { + if (!baseFileOnly) { row[idx++] = fs.getLogFiles().count(); row[idx++] = fs.getLogFiles().mapToLong(HoodieLogFile::getFileSize).sum(); long logFilesScheduledForCompactionTotalSize = @@ -190,7 +190,7 @@ public class FileSystemViewCommand implements CommandMarker { entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString()))); Map> fieldNameToConverterMap = new HashMap<>(); fieldNameToConverterMap.put("Data-File Size", converterFunction); - if (!readOptimizedOnly) { + if (!baseFileOnly) { fieldNameToConverterMap.put("Total Delta Size", converterFunction); fieldNameToConverterMap.put("Delta Size - compaction scheduled", converterFunction); fieldNameToConverterMap.put("Delta Size - compaction unscheduled", converterFunction); @@ -199,7 +199,7 @@ public class FileSystemViewCommand implements CommandMarker { TableHeader header = new TableHeader().addTableHeaderField("Partition").addTableHeaderField("FileId") .addTableHeaderField("Base-Instant").addTableHeaderField("Data-File").addTableHeaderField("Data-File Size"); - if (!readOptimizedOnly) { + if (!baseFileOnly) { header = header.addTableHeaderField("Num Delta Files").addTableHeaderField("Total Delta Size") .addTableHeaderField("Delta Size - compaction scheduled") .addTableHeaderField("Delta Size - compaction unscheduled") @@ -216,14 +216,14 @@ public class FileSystemViewCommand implements CommandMarker { * * @param globRegex Path Regex * @param maxInstant Max Instants to be used for displaying file-instants - * @param readOptimizedOnly Include only read optimized view + * @param basefileOnly Include only base file view * @param includeMaxInstant Include Max instant * @param includeInflight Include inflight instants * @param excludeCompaction Exclude Compaction instants * @return * @throws IOException */ - private HoodieTableFileSystemView buildFileSystemView(String globRegex, String maxInstant, boolean readOptimizedOnly, + private HoodieTableFileSystemView buildFileSystemView(String globRegex, String maxInstant, boolean basefileOnly, boolean includeMaxInstant, boolean includeInflight, boolean excludeCompaction) throws IOException { HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); HoodieTableMetaClient metaClient = @@ -234,7 +234,7 @@ public class FileSystemViewCommand implements CommandMarker { Stream instantsStream; HoodieTimeline timeline; - if (readOptimizedOnly) { + if (basefileOnly) { timeline = metaClient.getActiveTimeline().getCommitTimeline(); } else if (excludeCompaction) { timeline = metaClient.getActiveTimeline().getCommitsTimeline(); diff --git a/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala b/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala index f40501a91..d7b196d8c 100644 --- a/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala +++ b/hudi-cli/src/main/scala/org/apache/hudi/cli/DedupeSparkJob.scala @@ -20,7 +20,7 @@ package org.apache.hudi.cli import java.util.stream.Collectors import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} -import org.apache.hudi.common.model.{HoodieDataFile, HoodieRecord} +import org.apache.hudi.common.model.{HoodieBaseFile, HoodieRecord} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.common.util.FSUtils @@ -80,7 +80,7 @@ class DedupeSparkJob(basePath: String, val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"$basePath/$duplicatedPartitionPath")) val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) - val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]()) + val latestFiles: java.util.List[HoodieBaseFile] = fsView.getLatestBaseFiles().collect(Collectors.toList[HoodieBaseFile]()) val filteredStatuses = latestFiles.map(f => f.getPath) LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}") @@ -132,7 +132,7 @@ class DedupeSparkJob(basePath: String, val allFiles = fs.listStatus(new Path(s"$basePath/$duplicatedPartitionPath")) val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) - val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]()) + val latestFiles: java.util.List[HoodieBaseFile] = fsView.getLatestBaseFiles().collect(Collectors.toList[HoodieBaseFile]()) val fileNameToPathMap = latestFiles.map(f => (f.getFileId, new Path(f.getPath))).toMap val dupeFixPlan = planDuplicateFix() diff --git a/hudi-client/src/main/java/org/apache/hudi/CompactionAdminClient.java b/hudi-client/src/main/java/org/apache/hudi/CompactionAdminClient.java index 56a47b732..00f65a3b7 100644 --- a/hudi-client/src/main/java/org/apache/hudi/CompactionAdminClient.java +++ b/hudi-client/src/main/java/org/apache/hudi/CompactionAdminClient.java @@ -23,7 +23,7 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -293,7 +293,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { .filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst()); if (fileSliceOptional.isPresent()) { FileSlice fs = fileSliceOptional.get(); - Option df = fs.getDataFile(); + Option df = fs.getBaseFile(); if (operation.getDataFileName().isPresent()) { String expPath = metaClient.getFs() .getFileStatus( @@ -448,7 +448,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { .orElse(HoodieLogFile.LOGFILE_BASE_VERSION - 1); String logExtn = fileSliceForCompaction.getLogFiles().findFirst().map(lf -> "." + lf.getFileExtension()) .orElse(HoodieLogFile.DELTA_EXTENSION); - String parentPath = fileSliceForCompaction.getDataFile().map(df -> new Path(df.getPath()).getParent().toString()) + String parentPath = fileSliceForCompaction.getBaseFile().map(df -> new Path(df.getPath()).getParent().toString()) .orElse(fileSliceForCompaction.getLogFiles().findFirst().map(lf -> lf.getPath().getParent().toString()).get()); for (HoodieLogFile toRepair : logFilesToRepair) { int version = maxUsedVersion + 1; diff --git a/hudi-client/src/main/java/org/apache/hudi/HoodieReadClient.java b/hudi-client/src/main/java/org/apache/hudi/HoodieReadClient.java index ba3d9b9d6..ed4ade7fc 100644 --- a/hudi-client/src/main/java/org/apache/hudi/HoodieReadClient.java +++ b/hudi-client/src/main/java/org/apache/hudi/HoodieReadClient.java @@ -20,7 +20,7 @@ package org.apache.hudi; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.embedded.EmbeddedTimelineService; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -136,8 +136,8 @@ public class HoodieReadClient extends AbstractHoo private Option convertToDataFilePath(Option> partitionPathFileIDPair) { if (partitionPathFileIDPair.isPresent()) { - HoodieDataFile dataFile = hoodieTable.getROFileSystemView() - .getLatestDataFile(partitionPathFileIDPair.get().getLeft(), partitionPathFileIDPair.get().getRight()).get(); + HoodieBaseFile dataFile = hoodieTable.getBaseFileOnlyView() + .getLatestBaseFile(partitionPathFileIDPair.get().getLeft(), partitionPathFileIDPair.get().getRight()).get(); return Option.of(dataFile.getPath()); } else { return Option.empty(); diff --git a/hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java b/hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java index 0b8df71f2..c88a30264 100644 --- a/hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java +++ b/hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java @@ -26,7 +26,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -34,7 +34,7 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; -import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView; +import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; @@ -592,9 +592,9 @@ public class HoodieWriteClient extends AbstractHo .mapToPair((PairFunction>) partitionPath -> { // Scan all partitions files with this commit time LOG.info("Collecting latest files in partition path " + partitionPath); - ReadOptimizedView view = table.getROFileSystemView(); - List latestFiles = view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime) - .map(HoodieDataFile::getFileName).collect(Collectors.toList()); + BaseFileOnlyView view = table.getBaseFileOnlyView(); + List latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, commitTime) + .map(HoodieBaseFile::getFileName).collect(Collectors.toList()); return new Tuple2<>(partitionPath, latestFiles); }).collectAsMap(); diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java index 03fed3439..d95a89129 100644 --- a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java +++ b/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java @@ -245,8 +245,8 @@ public class HoodieBloomIndex extends HoodieIndex hoodieTable.getMetaClient().getCommitsTimeline().filterCompletedInstants().lastInstant(); List> filteredFiles = new ArrayList<>(); if (latestCommitTime.isPresent()) { - filteredFiles = hoodieTable.getROFileSystemView() - .getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp()) + filteredFiles = hoodieTable.getBaseFileOnlyView() + .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp()) .map(f -> Pair.of(partitionPath, f.getFileId())).collect(toList()); } return filteredFiles.iterator(); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 7cd343f50..d37e11f36 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -29,7 +29,7 @@ import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; -import org.apache.hudi.common.table.TableFileSystemView.RealtimeView; +import org.apache.hudi.common.table.TableFileSystemView.SliceView; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; @@ -117,7 +117,7 @@ public class HoodieAppendHandle extends HoodieWri if (doInit) { this.partitionPath = record.getPartitionPath(); // extract some information from the first record - RealtimeView rtView = hoodieTable.getRTFileSystemView(); + SliceView rtView = hoodieTable.getSliceView(); Option fileSlice = rtView.getLatestFileSlice(partitionPath, fileId); // Set the base commit time as the current commitTime for new inserts into log files String baseInstantTime = instantTime; diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieCleanHelper.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieCleanHelper.java index 9c319c881..37c8d6dbb 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieCleanHelper.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieCleanHelper.java @@ -23,7 +23,7 @@ import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; @@ -76,7 +76,7 @@ public class HoodieCleanHelper> implements Seri this.commitTimeline = hoodieTable.getCompletedCommitTimeline(); this.config = config; this.fgIdToPendingCompactionOperations = - ((SyncableFileSystemView) hoodieTable.getRTFileSystemView()).getPendingCompactionOperations() + ((SyncableFileSystemView) hoodieTable.getSliceView()).getPendingCompactionOperations() .map(entry -> Pair.of( new HoodieFileGroupId(entry.getValue().getPartitionPath(), entry.getValue().getFileId()), entry.getValue())) @@ -148,7 +148,7 @@ public class HoodieCleanHelper> implements Seri while (fileSliceIterator.hasNext() && keepVersions > 0) { // Skip this most recent version FileSlice nextSlice = fileSliceIterator.next(); - Option dataFile = nextSlice.getDataFile(); + Option dataFile = nextSlice.getBaseFile(); if (dataFile.isPresent() && savepointedFiles.contains(dataFile.get().getFileName())) { // do not clean up a savepoint data file continue; @@ -158,8 +158,8 @@ public class HoodieCleanHelper> implements Seri // Delete the remaining files while (fileSliceIterator.hasNext()) { FileSlice nextSlice = fileSliceIterator.next(); - if (nextSlice.getDataFile().isPresent()) { - HoodieDataFile dataFile = nextSlice.getDataFile().get(); + if (nextSlice.getBaseFile().isPresent()) { + HoodieBaseFile dataFile = nextSlice.getBaseFile().get(); deletePaths.add(dataFile.getFileName()); } if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { @@ -212,7 +212,7 @@ public class HoodieCleanHelper> implements Seri // Ensure there are more than 1 version of the file (we only clean old files from updates) // i.e always spare the last commit. for (FileSlice aSlice : fileSliceList) { - Option aFile = aSlice.getDataFile(); + Option aFile = aSlice.getBaseFile(); String fileCommitTime = aSlice.getBaseInstantTime(); if (aFile.isPresent() && savepointedFiles.contains(aFile.get().getFileName())) { // do not clean up a savepoint data file diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java index 9f3bdbbdd..69ed578e5 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java @@ -19,7 +19,7 @@ package org.apache.hudi.io; import org.apache.hudi.common.bloom.filter.BloomFilter; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.util.HoodieTimer; @@ -113,7 +113,7 @@ public class HoodieKeyLookupHandle extends Hoodie LOG.debug("#The candidate row keys for " + partitionPathFilePair + " => " + candidateRecordKeys); } - HoodieDataFile dataFile = getLatestDataFile(); + HoodieBaseFile dataFile = getLatestDataFile(); List matchingKeys = checkCandidatesAgainstFile(hoodieTable.getHadoopConf(), candidateRecordKeys, new Path(dataFile.getPath())); LOG.info( diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index 630ba3741..c3d726ccb 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -19,7 +19,7 @@ package org.apache.hudi.io; import org.apache.hudi.WriteStatus; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -73,14 +73,14 @@ public class HoodieMergeHandle extends HoodieWrit Iterator> recordItr, String fileId) { super(config, commitTime, fileId, hoodieTable); String partitionPath = init(fileId, recordItr); - init(fileId, partitionPath, hoodieTable.getROFileSystemView().getLatestDataFile(partitionPath, fileId).get()); + init(fileId, partitionPath, hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get()); } /** * Called by compactor code path. */ public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, - Map> keyToNewRecords, String fileId, HoodieDataFile dataFileToBeMerged) { + Map> keyToNewRecords, String fileId, HoodieBaseFile dataFileToBeMerged) { super(config, commitTime, fileId, hoodieTable); this.keyToNewRecords = keyToNewRecords; this.useWriterSchema = true; @@ -154,7 +154,7 @@ public class HoodieMergeHandle extends HoodieWrit /** * Extract old file path, initialize StorageWriter and WriteStatus. */ - private void init(String fileId, String partitionPath, HoodieDataFile dataFileToBeMerged) { + private void init(String fileId, String partitionPath, HoodieBaseFile dataFileToBeMerged) { LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" + fileId); this.writtenRecordKeys = new HashSet<>(); writeStatus.setStat(new HoodieWriteStat()); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java index 322076657..e84b0fafb 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java @@ -18,7 +18,7 @@ package org.apache.hudi.io; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.ParquetUtils; import org.apache.hudi.common.util.collection.Pair; @@ -38,7 +38,7 @@ public class HoodieRangeInfoHandle extends Hoodie } public String[] getMinMaxKeys() { - HoodieDataFile dataFile = getLatestDataFile(); + HoodieBaseFile dataFile = getLatestDataFile(); return ParquetUtils.readMinMaxRecordKeys(hoodieTable.getHadoopConf(), new Path(dataFile.getPath())); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieReadHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieReadHandle.java index 2bfc46d53..6662d00ef 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieReadHandle.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieReadHandle.java @@ -18,7 +18,7 @@ package org.apache.hudi.io; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -52,8 +52,8 @@ public abstract class HoodieReadHandle extends Ho return partitionPathFilePair.getRight(); } - protected HoodieDataFile getLatestDataFile() { - return hoodieTable.getROFileSystemView() - .getLatestDataFile(partitionPathFilePair.getLeft(), partitionPathFilePair.getRight()).get(); + protected HoodieBaseFile getLatestDataFile() { + return hoodieTable.getBaseFileOnlyView() + .getLatestBaseFile(partitionPathFilePair.getLeft(), partitionPathFilePair.getRight()).get(); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/io/compact/HoodieRealtimeTableCompactor.java b/hudi-client/src/main/java/org/apache/hudi/io/compact/HoodieMergeOnReadTableCompactor.java similarity index 94% rename from hudi-client/src/main/java/org/apache/hudi/io/compact/HoodieRealtimeTableCompactor.java rename to hudi-client/src/main/java/org/apache/hudi/io/compact/HoodieMergeOnReadTableCompactor.java index 8ad6f870a..46a614832 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/compact/HoodieRealtimeTableCompactor.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/compact/HoodieMergeOnReadTableCompactor.java @@ -22,14 +22,14 @@ import org.apache.hudi.WriteStatus; import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.model.CompactionOperation; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; -import org.apache.hudi.common.table.TableFileSystemView.RealtimeView; +import org.apache.hudi.common.table.TableFileSystemView.SliceView; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.FSUtils; @@ -66,15 +66,15 @@ import java.util.stream.StreamSupport; import static java.util.stream.Collectors.toList; /** - * HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all possible compactions, + * Compacts a hoodie table with merge on read storage. Computes all possible compactions, * passes it through a CompactionFilter and executes all the compactions and writes a new version of base files and make * a normal commit * * @see HoodieCompactor */ -public class HoodieRealtimeTableCompactor implements HoodieCompactor { +public class HoodieMergeOnReadTableCompactor implements HoodieCompactor { - private static final Logger LOG = LogManager.getLogger(HoodieRealtimeTableCompactor.class); + private static final Logger LOG = LogManager.getLogger(HoodieMergeOnReadTableCompactor.class); // Accumulator to keep track of total log files for a table private AccumulatorV2 totalLogFiles; // Accumulator to keep track of total log file slices for a table @@ -128,7 +128,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { return Lists.newArrayList(); } - Option oldDataFileOpt = + Option oldDataFileOpt = operation.getBaseFile(metaClient.getBasePath(), operation.getPartitionPath()); // Compacting is very similar to applying updates to existing file @@ -170,7 +170,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { jsc.sc().register(totalFileSlices); Preconditions.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ, - "HoodieRealtimeTableCompactor can only compact table of type " + HoodieTableType.MERGE_ON_READ + " and not " + "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 @@ -188,7 +188,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { return null; } - RealtimeView fileSystemView = hoodieTable.getRTFileSystemView(); + SliceView fileSystemView = hoodieTable.getSliceView(); LOG.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); List operations = jsc.parallelize(partitionPaths, partitionPaths.size()) .flatMap((FlatMapFunction) partitionPath -> fileSystemView @@ -201,7 +201,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { // Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO // for spark Map operations and collecting them finally in Avro generated classes for storing // into meta files. - Option dataFile = s.getDataFile(); + Option dataFile = s.getBaseFile(); return new CompactionOperation(dataFile, partitionPath, logFiles, config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles)); }).filter(c -> !c.getDeltaFileNames().isEmpty()).collect(toList()).iterator()) diff --git a/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/CompactionStrategy.java b/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/CompactionStrategy.java index 79a14b64c..4c0311658 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/CompactionStrategy.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/CompactionStrategy.java @@ -20,13 +20,13 @@ package org.apache.hudi.io.compact.strategy; import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.FSUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor; +import org.apache.hudi.io.compact.HoodieMergeOnReadTableCompactor; import com.google.common.collect.Maps; @@ -40,7 +40,7 @@ import java.util.Map; * compaction operation to run in a single compaction. Implementation of CompactionStrategy cannot hold any state. * Difference instantiations can be passed in every time * - * @see HoodieRealtimeTableCompactor + * @see HoodieMergeOnReadTableCompactor */ public abstract class CompactionStrategy implements Serializable { @@ -59,7 +59,7 @@ public abstract class CompactionStrategy implements Serializable { * @param logFiles - List of log files to compact with the base file * @return Map[String, Object] - metrics captured */ - public Map captureMetrics(HoodieWriteConfig writeConfig, Option dataFile, + public Map captureMetrics(HoodieWriteConfig writeConfig, Option dataFile, String partitionPath, List logFiles) { Map metrics = Maps.newHashMap(); Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize(); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java index 11c2752ce..4a78c75a4 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java @@ -20,7 +20,7 @@ package org.apache.hudi.io.compact.strategy; import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -43,7 +43,7 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE"; @Override - public Map captureMetrics(HoodieWriteConfig config, Option dataFile, + public Map captureMetrics(HoodieWriteConfig config, Option dataFile, String partitionPath, List logFiles) { Map metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles); diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java index f1f277bad..9690e4446 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java @@ -26,7 +26,7 @@ import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -84,7 +84,9 @@ import java.util.stream.Collectors; import scala.Tuple2; /** - * Implementation of a very heavily read-optimized Hoodie Table where. + * Implementation of a very heavily read-optimized Hoodie Table where, all data is stored in base files, with + * zero read amplification. + * *

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

@@ -181,7 +183,7 @@ public class HoodieCopyOnWriteTable extends Hoodi } public Iterator> handleUpdate(String commitTime, String fileId, - Map> keyToNewRecords, HoodieDataFile oldDataFile) throws IOException { + Map> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException { // these are updates HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, keyToNewRecords, oldDataFile); return handleUpdateInternal(upsertHandle, commitTime, fileId); @@ -223,7 +225,7 @@ public class HoodieCopyOnWriteTable extends Hoodi } protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId, - Map> keyToNewRecords, HoodieDataFile dataFileToBeMerged) { + Map> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) { return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileId, dataFileToBeMerged); } @@ -685,10 +687,10 @@ public class HoodieCopyOnWriteTable extends Hoodi if (!commitTimeline.empty()) { // if we have some commits HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); - List allFiles = getROFileSystemView() - .getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList()); + List allFiles = getBaseFileOnlyView() + .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList()); - for (HoodieDataFile file : allFiles) { + for (HoodieBaseFile file : allFiles) { if (file.getFileSize() < config.getParquetSmallFileLimit()) { String filename = file.getFileName(); SmallFile sf = new SmallFile(); diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieMergeOnReadTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieMergeOnReadTable.java index a654fcbf2..754b0ac8a 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieMergeOnReadTable.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/HoodieMergeOnReadTable.java @@ -40,7 +40,7 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.func.MergeOnReadLazyInsertIterable; import org.apache.hudi.io.HoodieAppendHandle; -import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor; +import org.apache.hudi.io.compact.HoodieMergeOnReadTableCompactor; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -61,7 +61,8 @@ import java.util.Objects; import java.util.stream.Collectors; /** - * Implementation of a more real-time read-optimized Hoodie Table where + * Implementation of a more real-time Hoodie Table the provides tradeoffs on read and write cost/amplification. + * *

* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the * smallest existing file, to expand it @@ -142,10 +143,10 @@ public class HoodieMergeOnReadTable extends Hoodi } LOG.info("Compacting merge on read table " + config.getBasePath()); - HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor(); + HoodieMergeOnReadTableCompactor compactor = new HoodieMergeOnReadTableCompactor(); try { return compactor.generateCompactionPlan(jsc, this, config, instantTime, - ((SyncableFileSystemView) getRTFileSystemView()).getPendingCompactionOperations() + ((SyncableFileSystemView) getSliceView()).getPendingCompactionOperations() .map(instantTimeCompactionopPair -> instantTimeCompactionopPair.getValue().getFileGroupId()) .collect(Collectors.toSet())); @@ -157,7 +158,7 @@ public class HoodieMergeOnReadTable extends Hoodi @Override public JavaRDD compact(JavaSparkContext jsc, String compactionInstantTime, HoodieCompactionPlan compactionPlan) { - HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor(); + HoodieMergeOnReadTableCompactor compactor = new HoodieMergeOnReadTableCompactor(); try { return compactor.compact(jsc, compactionPlan, this, config, compactionInstantTime); } catch (IOException e) { @@ -344,10 +345,10 @@ public class HoodieMergeOnReadTable extends Hoodi if (!index.canIndexLogFiles()) { // TODO : choose last N small files since there can be multiple small files written to a single partition // by different spark partitions in a single batch - Option smallFileSlice = Option.fromJavaOptional(getRTFileSystemView() + Option smallFileSlice = Option.fromJavaOptional(getSliceView() .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false) - .filter(fileSlice -> fileSlice.getLogFiles().count() < 1 && fileSlice.getDataFile().get().getFileSize() < config.getParquetSmallFileLimit()) - .min((FileSlice left, FileSlice right) -> left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() ? -1 : 1)); + .filter(fileSlice -> fileSlice.getLogFiles().count() < 1 && fileSlice.getBaseFile().get().getFileSize() < config.getParquetSmallFileLimit()) + .min((FileSlice left, FileSlice right) -> left.getBaseFile().get().getFileSize() < right.getBaseFile().get().getFileSize() ? -1 : 1)); if (smallFileSlice.isPresent()) { allSmallFileSlices.add(smallFileSlice.get()); } @@ -355,7 +356,7 @@ public class HoodieMergeOnReadTable extends Hoodi // If we can index log files, we can add more inserts to log files for fileIds including those under // pending compaction. List allFileSlices = - getRTFileSystemView().getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), true) + getSliceView().getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), true) .collect(Collectors.toList()); for (FileSlice fileSlice : allFileSlices) { if (isSmallFile(fileSlice)) { @@ -366,9 +367,9 @@ public class HoodieMergeOnReadTable extends Hoodi // Create SmallFiles from the eligible file slices for (FileSlice smallFileSlice : allSmallFileSlices) { SmallFile sf = new SmallFile(); - if (smallFileSlice.getDataFile().isPresent()) { + if (smallFileSlice.getBaseFile().isPresent()) { // TODO : Move logic of file name, file id, base commit time handling inside file slice - String filename = smallFileSlice.getDataFile().get().getFileName(); + String filename = smallFileSlice.getBaseFile().get().getFileName(); sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename)); sf.sizeBytes = getTotalFileSize(smallFileSlice); smallFileLocations.add(sf); @@ -394,10 +395,10 @@ public class HoodieMergeOnReadTable extends Hoodi } private long getTotalFileSize(FileSlice fileSlice) { - if (!fileSlice.getDataFile().isPresent()) { + if (!fileSlice.getBaseFile().isPresent()) { return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList())); } else { - return fileSlice.getDataFile().get().getFileSize() + return fileSlice.getBaseFile().get().getFileSize() + convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList())); } } @@ -428,7 +429,7 @@ public class HoodieMergeOnReadTable extends Hoodi // used to write the new log files. In this case, the commit time for the log file is the compaction requested time. // But the index (global) might store the baseCommit of the parquet and not the requested, hence get the // baseCommit always by listing the file slice - Map fileIdToBaseCommitTimeForLogMap = this.getRTFileSystemView().getLatestFileSlices(partitionPath) + Map fileIdToBaseCommitTimeForLogMap = this.getSliceView().getLatestFileSlices(partitionPath) .collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime)); return commitMetadata.getPartitionToWriteStats().get(partitionPath).stream().filter(wStat -> { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java index d2f571564..2762048eb 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -33,6 +33,8 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.SyncableFileSystemView; import org.apache.hudi.common.table.TableFileSystemView; +import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView; +import org.apache.hudi.common.table.TableFileSystemView.SliceView; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.FileSystemViewManager; @@ -145,16 +147,16 @@ public abstract class HoodieTable implements Seri } /** - * Get the read optimized view of the file system for this table. + * Get the base file only view of the file system for this table. */ - public TableFileSystemView.ReadOptimizedView getROFileSystemView() { + public BaseFileOnlyView getBaseFileOnlyView() { return getViewManager().getFileSystemView(metaClient.getBasePath()); } /** - * Get the real time view of the file system for this table. + * Get the full view of the file system for this table. */ - public TableFileSystemView.RealtimeView getRTFileSystemView() { + public SliceView getSliceView() { return getViewManager().getFileSystemView(metaClient.getBasePath()); } diff --git a/hudi-client/src/test/java/org/apache/hudi/TestAsyncCompaction.java b/hudi-client/src/test/java/org/apache/hudi/TestAsyncCompaction.java index d3d79545e..4f8fbb1f4 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestAsyncCompaction.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestAsyncCompaction.java @@ -23,7 +23,7 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.HoodieClientTestUtils; import org.apache.hudi.common.HoodieTestDataGenerator; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -402,7 +402,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase { assertEquals("Expect baseInstant to match compaction Instant", fileSlice.getBaseInstantTime(), opPair.getKey()); assertTrue("Expect atleast one log file to be present where the latest delta commit was written", fileSlice.getLogFiles().count() > 0); - assertFalse("Expect no data-file to be present", fileSlice.getDataFile().isPresent()); + assertFalse("Expect no data-file to be present", fileSlice.getBaseFile().isPresent()); } else { assertTrue("Expect baseInstant to be less than or equal to latestDeltaCommit", fileSlice.getBaseInstantTime().compareTo(latestDeltaCommit) <= 0); @@ -439,8 +439,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase { assertNoWriteErrors(statusList); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = getHoodieTable(metaClient, cfg); - List dataFilesToRead = getCurrentLatestDataFiles(hoodieTable, cfg); - assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", + List dataFilesToRead = getCurrentLatestDataFiles(hoodieTable, cfg); + assertTrue("should list the parquet files we wrote in the delta commit", dataFilesToRead.stream().findAny().isPresent()); validateDeltaCommit(firstInstant, fgIdToCompactionOperation, cfg); } @@ -487,7 +487,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase { assertFalse("Verify all file-slices have base-instant same as compaction instant", fileSliceList.stream() .anyMatch(fs -> !fs.getBaseInstantTime().equals(compactionInstantTime))); assertFalse("Verify all file-slices have data-files", - fileSliceList.stream().anyMatch(fs -> !fs.getDataFile().isPresent())); + fileSliceList.stream().anyMatch(fs -> !fs.getBaseFile().isPresent())); if (hasDeltaCommitAfterPendingCompaction) { assertFalse("Verify all file-slices have atleast one log-file", @@ -533,11 +533,11 @@ public class TestAsyncCompaction extends TestHoodieClientBase { return statusList; } - private List getCurrentLatestDataFiles(HoodieTable table, HoodieWriteConfig cfg) throws IOException { + private List getCurrentLatestDataFiles(HoodieTable table, HoodieWriteConfig cfg) throws IOException { FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(table.getMetaClient().getFs(), cfg.getBasePath()); HoodieTableFileSystemView view = new HoodieTableFileSystemView(table.getMetaClient(), table.getCompletedCommitsTimeline(), allFiles); - return view.getLatestDataFiles().collect(Collectors.toList()); + return view.getLatestBaseFiles().collect(Collectors.toList()); } private List getCurrentLatestFileSlices(HoodieTable table) { diff --git a/hudi-client/src/test/java/org/apache/hudi/TestCleaner.java b/hudi-client/src/test/java/org/apache/hudi/TestCleaner.java index baec87508..24aa9cd1b 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestCleaner.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestCleaner.java @@ -26,7 +26,7 @@ import org.apache.hudi.common.HoodieTestDataGenerator; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; @@ -267,8 +267,8 @@ public class TestCleaner extends TestHoodieClientBase { for (HoodieFileGroup fileGroup : fileGroups) { if (compactionFileIdToLatestFileSlice.containsKey(fileGroup.getFileGroupId())) { // Ensure latest file-slice selected for compaction is retained - Option dataFileForCompactionPresent = - Option.fromJavaOptional(fileGroup.getAllDataFiles().filter(df -> { + Option dataFileForCompactionPresent = + Option.fromJavaOptional(fileGroup.getAllBaseFiles().filter(df -> { return compactionFileIdToLatestFileSlice.get(fileGroup.getFileGroupId()).getBaseInstantTime() .equals(df.getCommitTime()); }).findAny()); @@ -277,7 +277,7 @@ public class TestCleaner extends TestHoodieClientBase { } else { // file has no more than max versions String fileId = fileGroup.getFileGroupId().getFileId(); - List dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList()); + List dataFiles = fileGroup.getAllBaseFiles().collect(Collectors.toList()); assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions", dataFiles.size() <= maxVersions); @@ -391,7 +391,7 @@ public class TestCleaner extends TestHoodieClientBase { List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { Set commitTimes = new HashSet<>(); - fileGroup.getAllDataFiles().forEach(value -> { + fileGroup.getAllBaseFiles().forEach(value -> { LOG.debug("Data File - " + value); commitTimes.add(value.getCommitTime()); }); @@ -1025,7 +1025,7 @@ public class TestCleaner extends TestHoodieClientBase { metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); FileSlice slice = - table.getRTFileSystemView().getLatestFileSlices(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) + table.getSliceView().getLatestFileSlices(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) .filter(fs -> fs.getFileId().equals(fileId)).findFirst().get(); List slices = new ArrayList<>(); if (compactionInstantsToFileSlices.containsKey(compactionInstants[j])) { @@ -1069,12 +1069,12 @@ public class TestCleaner extends TestHoodieClientBase { expFileIdToPendingCompaction.forEach((fileId, value) -> { String baseInstantForCompaction = fileIdToLatestInstantBeforeCompaction.get(fileId); - Option fileSliceForCompaction = Option.fromJavaOptional(hoodieTable.getRTFileSystemView() + Option fileSliceForCompaction = Option.fromJavaOptional(hoodieTable.getSliceView() .getLatestFileSlicesBeforeOrOn(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, baseInstantForCompaction, true) .filter(fs -> fs.getFileId().equals(fileId)).findFirst()); Assert.assertTrue("Base Instant for Compaction must be preserved", fileSliceForCompaction.isPresent()); - Assert.assertTrue("FileSlice has data-file", fileSliceForCompaction.get().getDataFile().isPresent()); + Assert.assertTrue("FileSlice has data-file", fileSliceForCompaction.get().getBaseFile().isPresent()); Assert.assertEquals("FileSlice has log-files", 2, fileSliceForCompaction.get().getLogFiles().count()); }); @@ -1135,9 +1135,9 @@ public class TestCleaner extends TestHoodieClientBase { private Stream> convertPathToFileIdWithCommitTime(final HoodieTableMetaClient metaClient, List paths) { Predicate roFilePredicate = - path -> path.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()); + path -> path.contains(metaClient.getTableConfig().getBaseFileFormat().getFileExtension()); Predicate rtFilePredicate = - path -> path.contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension()); + path -> path.contains(metaClient.getTableConfig().getLogFileFormat().getFileExtension()); Stream> stream1 = paths.stream().filter(roFilePredicate).map(fullPath -> { String fileName = Paths.get(fullPath).getFileName().toString(); return Pair.of(FSUtils.getFileId(fileName), FSUtils.getCommitTime(fileName)); diff --git a/hudi-client/src/test/java/org/apache/hudi/TestClientRollback.java b/hudi-client/src/test/java/org/apache/hudi/TestClientRollback.java index abefe86e5..0da3959a8 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestClientRollback.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestClientRollback.java @@ -20,11 +20,11 @@ package org.apache.hudi; import org.apache.hudi.common.HoodieTestDataGenerator; import org.apache.hudi.common.model.HoodieCleaningPolicy; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTestUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView; +import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.FSUtils; import org.apache.hudi.config.HoodieCompactionConfig; @@ -100,15 +100,15 @@ public class TestClientRollback extends TestHoodieClientBase { FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); - final ReadOptimizedView view1 = table.getROFileSystemView(); + final BaseFileOnlyView view1 = table.getBaseFileOnlyView(); - List dataFiles = partitionPaths.stream().flatMap(s -> { - return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003")); + List dataFiles = partitionPaths.stream().flatMap(s -> { + return view1.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("003")); }).collect(Collectors.toList()); assertEquals("The data files for commit 003 should be present", 3, dataFiles.size()); dataFiles = partitionPaths.stream().flatMap(s -> { - return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); + return view1.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("002")); }).collect(Collectors.toList()); assertEquals("The data files for commit 002 should be present", 3, dataFiles.size()); @@ -125,9 +125,9 @@ public class TestClientRollback extends TestHoodieClientBase { metaClient = HoodieTableMetaClient.reload(metaClient); table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); - final ReadOptimizedView view2 = table.getROFileSystemView(); + final BaseFileOnlyView view2 = table.getBaseFileOnlyView(); - dataFiles = partitionPaths.stream().flatMap(s -> view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"))).collect(Collectors.toList()); + dataFiles = partitionPaths.stream().flatMap(s -> view2.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("004"))).collect(Collectors.toList()); assertEquals("The data files for commit 004 should be present", 3, dataFiles.size()); // rolling back to a non existent savepoint must not succeed @@ -144,19 +144,19 @@ public class TestClientRollback extends TestHoodieClientBase { metaClient = HoodieTableMetaClient.reload(metaClient); table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc); - final ReadOptimizedView view3 = table.getROFileSystemView(); + final BaseFileOnlyView view3 = table.getBaseFileOnlyView(); dataFiles = partitionPaths.stream().flatMap(s -> { - return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); + return view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("002")); }).collect(Collectors.toList()); assertEquals("The data files for commit 002 be available", 3, dataFiles.size()); dataFiles = partitionPaths.stream().flatMap(s -> { - return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003")); + return view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("003")); }).collect(Collectors.toList()); assertEquals("The data files for commit 003 should be rolled back", 0, dataFiles.size()); dataFiles = partitionPaths.stream().flatMap(s -> { - return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004")); + return view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("004")); }).collect(Collectors.toList()); assertEquals("The data files for commit 004 should be rolled back", 0, dataFiles.size()); } diff --git a/hudi-client/src/test/java/org/apache/hudi/TestCompactionAdminClient.java b/hudi-client/src/test/java/org/apache/hudi/TestCompactionAdminClient.java index 2ce452ad3..0ed435ce2 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestCompactionAdminClient.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestCompactionAdminClient.java @@ -275,7 +275,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { // Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true) .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)).forEach(fs -> { - Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent()); + Assert.assertFalse("No Data file must be present", fs.getBaseFile().isPresent()); Assert.assertEquals("No Log Files", 0, fs.getLogFiles().count()); }); @@ -336,7 +336,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true) .filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)) .filter(fs -> fs.getFileId().equals(op.getFileId())).forEach(fs -> { - Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent()); + Assert.assertFalse("No Data file must be present", fs.getBaseFile().isPresent()); Assert.assertEquals("No Log Files", 0, fs.getLogFiles().count()); }); diff --git a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientBase.java b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientBase.java index 2608aea99..91ca10b13 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientBase.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientBase.java @@ -157,7 +157,7 @@ public class TestHoodieClientBase extends HoodieClientTestHarness { protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) { HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); - ((SyncableFileSystemView) (table.getRTFileSystemView())).reset(); + ((SyncableFileSystemView) (table.getSliceView())).reset(); return table; } diff --git a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java index 4a8e518c0..25bd14b5b 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java @@ -21,7 +21,7 @@ package org.apache.hudi; import org.apache.hudi.common.HoodieClientTestUtils; import org.apache.hudi.common.HoodieTestDataGenerator; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRollingStat; @@ -30,7 +30,7 @@ import org.apache.hudi.common.model.HoodieTestUtils; import org.apache.hudi.common.model.TimelineLayoutVersion; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; -import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView; +import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.ConsistencyGuardConfig; @@ -510,12 +510,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = getHoodieTable(metadata, config); - ReadOptimizedView fileSystemView = table.getROFileSystemView(); - List files = - fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3).collect(Collectors.toList()); + BaseFileOnlyView fileSystemView = table.getBaseFileOnlyView(); + List files = + fileSystemView.getLatestBaseFilesBeforeOrOn(testPartitionPath, commitTime3).collect(Collectors.toList()); int numTotalInsertsInCommit3 = 0; int numTotalUpdatesInCommit3 = 0; - for (HoodieDataFile file : files) { + for (HoodieBaseFile file : files) { if (file.getFileName().contains(file1)) { assertEquals("Existing file should be expanded", commitTime3, file.getCommitTime()); records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath())); @@ -616,12 +616,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table = getHoodieTable(metaClient, config); - List files = table.getROFileSystemView() - .getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3).collect(Collectors.toList()); + List files = table.getBaseFileOnlyView() + .getLatestBaseFilesBeforeOrOn(testPartitionPath, commitTime3).collect(Collectors.toList()); assertEquals("Total of 2 valid data files", 2, files.size()); int totalInserts = 0; - for (HoodieDataFile file : files) { + for (HoodieBaseFile file : files) { assertEquals("All files must be at commit 3", commitTime3, file.getCommitTime()); records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath())); totalInserts += records.size(); diff --git a/hudi-client/src/test/java/org/apache/hudi/common/HoodieClientTestUtils.java b/hudi-client/src/test/java/org/apache/hudi/common/HoodieClientTestUtils.java index 06986533f..d13010649 100644 --- a/hudi-client/src/test/java/org/apache/hudi/common/HoodieClientTestUtils.java +++ b/hudi-client/src/test/java/org/apache/hudi/common/HoodieClientTestUtils.java @@ -25,13 +25,13 @@ import org.apache.hudi.common.bloom.filter.BloomFilter; import org.apache.hudi.common.bloom.filter.BloomFilterFactory; import org.apache.hudi.common.bloom.filter.BloomFilterTypeCode; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTestUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; -import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView; +import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.FSUtils; @@ -203,10 +203,10 @@ public class HoodieClientTestUtils { try { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); for (String path : paths) { - ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(metaClient, + BaseFileOnlyView fileSystemView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path))); - List latestFiles = fileSystemView.getLatestDataFiles().collect(Collectors.toList()); - for (HoodieDataFile file : latestFiles) { + List latestFiles = fileSystemView.getLatestBaseFiles().collect(Collectors.toList()); + for (HoodieBaseFile file : latestFiles) { filteredPaths.add(file.getPath()); } } diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCompactor.java b/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCompactor.java index b986e8a17..01dc5425c 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCompactor.java +++ b/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieCompactor.java @@ -157,7 +157,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { table = HoodieTable.getHoodieTable(metaClient, config, jsc); for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = - table.getRTFileSystemView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); + table.getSliceView().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()); } @@ -185,7 +185,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness { return HoodieTableType.MERGE_ON_READ; } - // TODO - after modifying HoodieReadClient to support realtime tables - add more tests to make + // TODO - after modifying HoodieReadClient to support mor 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/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieDataFile.java b/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieBaseFile.java similarity index 83% rename from hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieDataFile.java rename to hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieBaseFile.java index 8d7851362..c23cfe223 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieDataFile.java +++ b/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieBaseFile.java @@ -18,21 +18,21 @@ package org.apache.hudi.io.strategy; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import java.util.UUID; -public class TestHoodieDataFile extends HoodieDataFile { +public class TestHoodieBaseFile extends HoodieBaseFile { private final long size; - public TestHoodieDataFile(long size) { + public TestHoodieBaseFile(long size) { super("/tmp/XYXYXYXYXYYX_11_20180918020003.parquet"); this.size = size; } - public static HoodieDataFile newDataFile(long size) { - return new TestHoodieDataFile(size); + public static HoodieBaseFile newDataFile(long size) { + return new TestHoodieBaseFile(size); } @Override diff --git a/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java b/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java index 5eda0b2c9..95be5a953 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java +++ b/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java @@ -19,7 +19,7 @@ package org.apache.hudi.io.strategy; import org.apache.hudi.avro.model.HoodieCompactionOperation; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -239,7 +239,7 @@ public class TestHoodieCompactionStrategy { List operations = new ArrayList<>(sizesMap.size()); sizesMap.forEach((k, v) -> { - HoodieDataFile df = TestHoodieDataFile.newDataFile(k); + HoodieBaseFile df = TestHoodieBaseFile.newDataFile(k); String partitionPath = keyToPartitionMap.get(k); List logFiles = v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()); operations.add(new HoodieCompactionOperation(df.getCommitTime(), diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestMergeOnReadTable.java b/hudi-client/src/test/java/org/apache/hudi/table/TestMergeOnReadTable.java index 840dbed31..c68413e60 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/TestMergeOnReadTable.java +++ b/hudi-client/src/test/java/org/apache/hudi/table/TestMergeOnReadTable.java @@ -28,7 +28,7 @@ import org.apache.hudi.common.HoodieTestDataGenerator; import org.apache.hudi.common.TestRawTripPayload.MetadataMergeWriteStatus; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -39,8 +39,8 @@ import org.apache.hudi.common.model.HoodieTestUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.SyncableFileSystemView; -import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView; -import org.apache.hudi.common.table.TableFileSystemView.RealtimeView; +import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView; +import org.apache.hudi.common.table.TableFileSystemView.SliceView; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; @@ -130,14 +130,14 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { assertFalse(commit.isPresent()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); - ReadOptimizedView roView = + BaseFileOnlyView roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); - Stream dataFilesToRead = roView.getLatestDataFiles(); + Stream dataFilesToRead = roView.getLatestBaseFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); - dataFilesToRead = roView.getLatestDataFiles(); - assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", + dataFilesToRead = roView.getLatestBaseFiles(); + assertTrue("should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); /** @@ -170,7 +170,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); - dataFilesToRead = roView.getLatestDataFiles(); + dataFilesToRead = roView.getLatestBaseFiles(); assertTrue(dataFilesToRead.findAny().isPresent()); // verify that there is a commit @@ -238,14 +238,14 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { assertFalse(commit.isPresent()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); - ReadOptimizedView roView = + BaseFileOnlyView roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); - Stream dataFilesToRead = roView.getLatestDataFiles(); + Stream dataFilesToRead = roView.getLatestBaseFiles(); assertFalse(dataFilesToRead.findAny().isPresent()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); - dataFilesToRead = roView.getLatestDataFiles(); - assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", + dataFilesToRead = roView.getLatestBaseFiles(); + assertTrue("should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); /** @@ -281,10 +281,10 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); - dataFilesToRead = roView.getLatestDataFiles(); + dataFilesToRead = roView.getLatestBaseFiles(); assertTrue(dataFilesToRead.findAny().isPresent()); - List dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); + List dataFiles = roView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); // Wrote 20 records and deleted 20 records, so remaining 20-20 = 0 assertEquals("Must contain 0 records", 0, recordsRead.size()); @@ -343,7 +343,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); final String absentCommit = newCommitTime; - assertFalse(roView.getLatestDataFiles().anyMatch(file -> absentCommit.equals(file.getCommitTime()))); + assertFalse(roView.getLatestBaseFiles().anyMatch(file -> absentCommit.equals(file.getCommitTime()))); } } @@ -379,14 +379,14 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { assertFalse(commit.isPresent()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); - ReadOptimizedView roView = + BaseFileOnlyView roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); - Stream dataFilesToRead = roView.getLatestDataFiles(); + Stream dataFilesToRead = roView.getLatestBaseFiles(); assertTrue(!dataFilesToRead.findAny().isPresent()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); - dataFilesToRead = roView.getLatestDataFiles(); - assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", + dataFilesToRead = roView.getLatestBaseFiles(); + assertTrue("should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); /** @@ -401,7 +401,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { copyOfRecords = dataGen.generateUpdates(commitTime1, copyOfRecords); copyOfRecords.addAll(dataGen.generateInserts(commitTime1, 200)); - List dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); + List dataFiles = roView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); @@ -415,7 +415,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { // After rollback, there should be no parquet file with the failed commit time Assert.assertEquals(Arrays.stream(allFiles) .filter(file -> file.getPath().getName().contains(commitTime1)).count(), 0); - dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); + dataFiles = roView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); } @@ -431,7 +431,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { copyOfRecords = dataGen.generateUpdates(commitTime2, copyOfRecords); copyOfRecords.addAll(dataGen.generateInserts(commitTime2, 200)); - List dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); + List dataFiles = roView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); @@ -452,7 +452,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); - dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); + dataFiles = roView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList()); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); // check that the number of records read is still correct after rollback operation assertEquals(recordsRead.size(), 200); @@ -483,7 +483,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get().getTimestamp(); - assertTrue(roView.getLatestDataFiles().anyMatch(file -> compactedCommitTime.equals(file.getCommitTime()))); + assertTrue(roView.getLatestBaseFiles().anyMatch(file -> compactedCommitTime.equals(file.getCommitTime()))); thirdClient.rollback(compactedCommitTime); @@ -491,7 +491,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles); - assertFalse(roView.getLatestDataFiles().anyMatch(file -> compactedCommitTime.equals(file.getCommitTime()))); + assertFalse(roView.getLatestBaseFiles().anyMatch(file -> compactedCommitTime.equals(file.getCommitTime()))); } } } @@ -526,14 +526,14 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { assertFalse(commit.isPresent()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); - ReadOptimizedView roView = + BaseFileOnlyView roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); - Stream dataFilesToRead = roView.getLatestDataFiles(); + Stream dataFilesToRead = roView.getLatestBaseFiles(); assertFalse(dataFilesToRead.findAny().isPresent()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); - dataFilesToRead = roView.getLatestDataFiles(); - assertTrue("ReadOptimizedTableView should list the parquet files we wrote in the delta commit", + dataFilesToRead = roView.getLatestBaseFiles(); + assertTrue("Should list the parquet files we wrote in the delta commit", dataFilesToRead.findAny().isPresent()); /** @@ -548,7 +548,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { copyOfRecords = dataGen.generateUpdates(newCommitTime, copyOfRecords); copyOfRecords.addAll(dataGen.generateInserts(newCommitTime, 200)); - List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); + List dataFiles = roView.getLatestBaseFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); assertEquals(recordsRead.size(), 200); @@ -611,7 +611,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { final String compactedCommitTime = metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get().getTimestamp(); - assertTrue(roView.getLatestDataFiles().anyMatch(file -> compactedCommitTime.equals(file.getCommitTime()))); + assertTrue(roView.getLatestBaseFiles().anyMatch(file -> compactedCommitTime.equals(file.getCommitTime()))); /** * Write 5 (updates) @@ -635,9 +635,9 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); - dataFilesToRead = roView.getLatestDataFiles(); + dataFilesToRead = roView.getLatestBaseFiles(); assertFalse(dataFilesToRead.findAny().isPresent()); - RealtimeView rtView = + SliceView rtView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); List fileGroups = ((HoodieTableFileSystemView) rtView).getAllFileGroups().collect(Collectors.toList()); @@ -689,16 +689,16 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { assertFalse(commit.isPresent()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); - ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient, + BaseFileOnlyView roView = new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline().filterCompletedInstants(), allFiles); - Stream dataFilesToRead = roView.getLatestDataFiles(); + Stream dataFilesToRead = roView.getLatestBaseFiles(); Map parquetFileIdToSize = - dataFilesToRead.collect(Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); + dataFilesToRead.collect(Collectors.toMap(HoodieBaseFile::getFileId, HoodieBaseFile::getFileSize)); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); - dataFilesToRead = roView.getLatestDataFiles(); - List dataFilesList = dataFilesToRead.collect(Collectors.toList()); - assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit", + dataFilesToRead = roView.getLatestBaseFiles(); + List dataFilesList = dataFilesToRead.collect(Collectors.toList()); + assertTrue("Should list the parquet files we wrote in the delta commit", dataFilesList.size() > 0); /** @@ -725,14 +725,14 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(), allFiles); - dataFilesToRead = roView.getLatestDataFiles(); - List newDataFilesList = dataFilesToRead.collect(Collectors.toList()); + dataFilesToRead = roView.getLatestBaseFiles(); + List newDataFilesList = dataFilesToRead.collect(Collectors.toList()); Map parquetFileIdToNewSize = - newDataFilesList.stream().collect(Collectors.toMap(HoodieDataFile::getFileId, HoodieDataFile::getFileSize)); + newDataFilesList.stream().collect(Collectors.toMap(HoodieBaseFile::getFileId, HoodieBaseFile::getFileSize)); assertTrue(parquetFileIdToNewSize.entrySet().stream().anyMatch(entry -> parquetFileIdToSize.get(entry.getKey()) < entry.getValue())); - List dataFiles = roView.getLatestDataFiles().map(HoodieDataFile::getPath).collect(Collectors.toList()); + List dataFiles = roView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList()); List recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath); // Wrote 20 records in 2 batches assertEquals("Must contain 40 records", 40, recordsRead.size()); @@ -770,11 +770,11 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); // In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state - ((SyncableFileSystemView) (table.getRTFileSystemView())).reset(); + ((SyncableFileSystemView) (table.getSliceView())).reset(); for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = - table.getRTFileSystemView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); + table.getSliceView().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()); } @@ -800,9 +800,9 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { for (String partitionPath : dataGen.getPartitionPaths()) { List groupedLogFiles = - table.getRTFileSystemView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); + table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); for (FileSlice slice : groupedLogFiles) { - assertEquals("After compaction there should be no log files visiable on a Realtime view", 0, slice.getLogFiles().count()); + assertEquals("After compaction there should be no log files visible on a full view", 0, slice.getLogFiles().count()); } List writeStatuses = result.collect(); assertTrue(writeStatuses.stream().anyMatch(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath))); @@ -827,12 +827,12 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc); - RealtimeView tableRTFileSystemView = table.getRTFileSystemView(); + SliceView tableRTFileSystemView = table.getSliceView(); long numLogFiles = 0; for (String partitionPath : dataGen.getPartitionPaths()) { assertEquals(0, tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getDataFile().isPresent()).count()); + .filter(fileSlice -> fileSlice.getBaseFile().isPresent()).count()); Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).anyMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath) .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count(); @@ -903,11 +903,11 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); - RealtimeView tableRTFileSystemView = table.getRTFileSystemView(); + SliceView tableRTFileSystemView = table.getSliceView(); long numLogFiles = 0; for (String partitionPath : dataGen.getPartitionPaths()) { - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getDataFile().isPresent())); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getBaseFile().isPresent())); Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath) .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count(); @@ -940,11 +940,11 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc); - RealtimeView tableRTFileSystemView = table.getRTFileSystemView(); + SliceView tableRTFileSystemView = table.getSliceView(); long numLogFiles = 0; for (String partitionPath : dataGen.getPartitionPaths()) { - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getDataFile().isPresent())); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getBaseFile().isPresent())); Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).anyMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath) .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count(); @@ -961,12 +961,12 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness { // Trigger a rollback of compaction writeClient.rollback(newCommitTime); table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc); - tableRTFileSystemView = table.getRTFileSystemView(); + tableRTFileSystemView = table.getSliceView(); ((SyncableFileSystemView) tableRTFileSystemView).reset(); Option lastInstant = ((SyncableFileSystemView) tableRTFileSystemView).getLastInstant(); System.out.println("Last Instant =" + lastInstant); for (String partitionPath : dataGen.getPartitionPaths()) { - Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getDataFile().isPresent())); + Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).noneMatch(fileSlice -> fileSlice.getBaseFile().isPresent())); Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).anyMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/CompactionOperation.java b/hudi-common/src/main/java/org/apache/hudi/common/model/CompactionOperation.java index ea178a832..dd4aaf40a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/CompactionOperation.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/CompactionOperation.java @@ -60,7 +60,7 @@ public class CompactionOperation implements Serializable { this.metrics = metrics; } - public CompactionOperation(Option dataFile, String partitionPath, List logFiles, + public CompactionOperation(Option dataFile, String partitionPath, List logFiles, Map metrics) { if (dataFile.isPresent()) { this.baseInstantTime = dataFile.get().getCommitTime(); @@ -111,9 +111,9 @@ public class CompactionOperation implements Serializable { return id; } - public Option getBaseFile(String basePath, String partitionPath) { + public Option getBaseFile(String basePath, String partitionPath) { Path dirPath = FSUtils.getPartitionPath(basePath, partitionPath); - return dataFileName.map(df -> new HoodieDataFile(new Path(dirPath, df).toString())); + return dataFileName.map(df -> new HoodieBaseFile(new Path(dirPath, df).toString())); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java b/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java index 7a6521464..19e62f999 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java @@ -44,7 +44,7 @@ public class FileSlice implements Serializable { /** * data file, with the compacted data, for this slice. */ - private HoodieDataFile dataFile; + private HoodieBaseFile baseFile; /** * List of appendable log files with real time data - Sorted with greater log version first - Always empty for @@ -59,12 +59,12 @@ public class FileSlice implements Serializable { public FileSlice(HoodieFileGroupId fileGroupId, String baseInstantTime) { this.fileGroupId = fileGroupId; this.baseInstantTime = baseInstantTime; - this.dataFile = null; + this.baseFile = null; this.logFiles = new TreeSet<>(HoodieLogFile.getReverseLogFileComparator()); } - public void setDataFile(HoodieDataFile dataFile) { - this.dataFile = dataFile; + public void setBaseFile(HoodieBaseFile baseFile) { + this.baseFile = baseFile; } public void addLogFile(HoodieLogFile logFile) { @@ -91,8 +91,8 @@ public class FileSlice implements Serializable { return fileGroupId; } - public Option getDataFile() { - return Option.ofNullable(dataFile); + public Option getBaseFile() { + return Option.ofNullable(baseFile); } public Option getLatestLogFile() { @@ -105,7 +105,7 @@ public class FileSlice implements Serializable { * @return */ public boolean isEmpty() { - return (dataFile == null) && (logFiles.isEmpty()); + return (baseFile == null) && (logFiles.isEmpty()); } @Override @@ -113,7 +113,7 @@ public class FileSlice implements Serializable { final StringBuilder sb = new StringBuilder("FileSlice {"); sb.append("fileGroupId=").append(fileGroupId); sb.append(", baseCommitTime=").append(baseInstantTime); - sb.append(", dataFile='").append(dataFile).append('\''); + sb.append(", baseFile='").append(baseFile).append('\''); sb.append(", logFiles='").append(logFiles).append('\''); sb.append('}'); return sb.toString(); @@ -129,7 +129,7 @@ public class FileSlice implements Serializable { } FileSlice slice = (FileSlice) o; return Objects.equals(fileGroupId, slice.fileGroupId) && Objects.equals(baseInstantTime, slice.baseInstantTime) - && Objects.equals(dataFile, slice.dataFile) && Objects.equals(logFiles, slice.logFiles); + && Objects.equals(baseFile, slice.baseFile) && Objects.equals(logFiles, slice.logFiles); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDataFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java similarity index 91% rename from hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDataFile.java rename to hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java index 4983b74d8..90e429b0d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDataFile.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java @@ -27,21 +27,21 @@ import java.io.Serializable; import java.util.Objects; /** - * Hoodie data file. + * Hoodie base file. */ -public class HoodieDataFile implements Serializable { +public class HoodieBaseFile implements Serializable { private transient FileStatus fileStatus; private final String fullPath; private long fileLen; - public HoodieDataFile(FileStatus fileStatus) { + public HoodieBaseFile(FileStatus fileStatus) { this.fileStatus = fileStatus; this.fullPath = fileStatus.getPath().toString(); this.fileLen = fileStatus.getLen(); } - public HoodieDataFile(String filePath) { + public HoodieBaseFile(String filePath) { this.fileStatus = null; this.fullPath = filePath; this.fileLen = -1; @@ -87,7 +87,7 @@ public class HoodieDataFile implements Serializable { if (o == null || getClass() != o.getClass()) { return false; } - HoodieDataFile dataFile = (HoodieDataFile) o; + HoodieBaseFile dataFile = (HoodieBaseFile) o; return Objects.equals(fullPath, dataFile.fullPath); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java index 3313586eb..83e38d4c3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java @@ -82,11 +82,11 @@ public class HoodieFileGroup implements Serializable { /** * Add a new datafile into the file group. */ - public void addDataFile(HoodieDataFile dataFile) { + public void addBaseFile(HoodieBaseFile dataFile) { if (!fileSlices.containsKey(dataFile.getCommitTime())) { fileSlices.put(dataFile.getCommitTime(), new FileSlice(fileGroupId, dataFile.getCommitTime())); } - fileSlices.get(dataFile.getCommitTime()).setDataFile(dataFile); + fileSlices.get(dataFile.getCommitTime()).setBaseFile(dataFile); } /** @@ -155,8 +155,8 @@ public class HoodieFileGroup implements Serializable { /** * Gets the latest data file. */ - public Option getLatestDataFile() { - return Option.fromJavaOptional(getAllDataFiles().findFirst()); + public Option getLatestDataFile() { + return Option.fromJavaOptional(getAllBaseFiles().findFirst()); } /** @@ -187,8 +187,8 @@ public class HoodieFileGroup implements Serializable { /** * Stream of committed data files, sorted reverse commit time. */ - public Stream getAllDataFiles() { - return getAllFileSlices().filter(slice -> slice.getDataFile().isPresent()).map(slice -> slice.getDataFile().get()); + public Stream getAllBaseFiles() { + return getAllFileSlices().filter(slice -> slice.getBaseFile().isPresent()).map(slice -> slice.getBaseFile().get()); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTableType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTableType.java index 6c24e391b..851771c08 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTableType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTableType.java @@ -27,9 +27,6 @@ package org.apache.hudi.common.model; *

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

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

- * SIMPLE_LSM - A simple 2 level LSM tree. */ public enum HoodieTableType { COPY_ON_WRITE, MERGE_ON_READ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index b19456536..49326b806 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -53,15 +53,19 @@ public class HoodieTableConfig implements Serializable { public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties"; public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name"; public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type"; + @Deprecated public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME = "hoodie.table.ro.file.format"; + @Deprecated public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME = "hoodie.table.rt.file.format"; + public static final String HOODIE_BASE_FILE_FORMAT_PROP_NAME = "hoodie.table.base.file.format"; + public static final String HOODIE_LOG_FILE_FORMAT_PROP_NAME = "hoodie.table.log.file.format"; public static final String HOODIE_TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version"; public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class"; public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = "hoodie.archivelog.folder"; public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE; - public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET; - public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG; + public static final HoodieFileFormat DEFAULT_BASE_FILE_FORMAT = HoodieFileFormat.PARQUET; + public static final HoodieFileFormat DEFAULT_LOG_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG; public static final String DEFAULT_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName(); public static final Integer DEFAULT_TIMELINE_LAYOUT_VERSION = TimelineLayoutVersion.VERSION_0; public static final String DEFAULT_ARCHIVELOG_FOLDER = ""; @@ -164,27 +168,33 @@ public class HoodieTableConfig implements Serializable { } /** - * Get the Read Optimized Storage Format. + * Get the base file storage format. * - * @return HoodieFileFormat for the Read Optimized Storage format + * @return HoodieFileFormat for the base file Storage format */ - public HoodieFileFormat getROFileFormat() { + public HoodieFileFormat getBaseFileFormat() { + if (props.containsKey(HOODIE_BASE_FILE_FORMAT_PROP_NAME)) { + return HoodieFileFormat.valueOf(props.getProperty(HOODIE_BASE_FILE_FORMAT_PROP_NAME)); + } if (props.containsKey(HOODIE_RO_FILE_FORMAT_PROP_NAME)) { return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RO_FILE_FORMAT_PROP_NAME)); } - return DEFAULT_RO_FILE_FORMAT; + return DEFAULT_BASE_FILE_FORMAT; } /** - * Get the Read Optimized Storage Format. + * Get the log Storage Format. * - * @return HoodieFileFormat for the Read Optimized Storage format + * @return HoodieFileFormat for the log Storage format */ - public HoodieFileFormat getRTFileFormat() { + public HoodieFileFormat getLogFileFormat() { + if (props.containsKey(HOODIE_LOG_FILE_FORMAT_PROP_NAME)) { + return HoodieFileFormat.valueOf(props.getProperty(HOODIE_LOG_FILE_FORMAT_PROP_NAME)); + } if (props.containsKey(HOODIE_RT_FILE_FORMAT_PROP_NAME)) { return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RT_FILE_FORMAT_PROP_NAME)); } - return DEFAULT_RT_FILE_FORMAT; + return DEFAULT_LOG_FILE_FORMAT; } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 1d27d81b7..40583ddc5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -299,7 +299,7 @@ public class HoodieTableMetaClient implements Serializable { } /** - * Helper method to initialize a given path, as a given storage type and table name. + * Helper method to initialize a given path, as a given type and table name. */ public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType, String tableName, String payloadClassName) throws IOException { @@ -437,7 +437,7 @@ public class HoodieTableMetaClient implements Serializable { case MERGE_ON_READ: return HoodieActiveTimeline.DELTA_COMMIT_ACTION; default: - throw new HoodieException("Could not commit on unknown storage type " + this.getTableType()); + throw new HoodieException("Could not commit on unknown table type " + this.getTableType()); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/SyncableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/SyncableFileSystemView.java index d8023a6a4..7a03f0ff6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/SyncableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/SyncableFileSystemView.java @@ -18,12 +18,15 @@ package org.apache.hudi.common.table; +import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView; +import org.apache.hudi.common.table.TableFileSystemView.SliceView; + /** - * A consolidated file-system view interface exposing both realtime and read-optimized views along with + * A consolidated file-system view interface exposing both complete slice and basefile only views along with * update operations. */ public interface SyncableFileSystemView - extends TableFileSystemView, TableFileSystemView.ReadOptimizedView, TableFileSystemView.RealtimeView { + extends TableFileSystemView, BaseFileOnlyView, SliceView { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableFileSystemView.java index f77d1e1c0..0a5bb3231 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableFileSystemView.java @@ -20,7 +20,7 @@ package org.apache.hudi.common.table; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; @@ -37,57 +37,57 @@ import java.util.stream.Stream; public interface TableFileSystemView { /** - * ReadOptimizedView with methods to only access latest version of file for the instant(s) passed. + * Methods to only access latest version of file for the instant(s) passed. */ - interface ReadOptimizedViewWithLatestSlice { + interface BaseFileOnlyViewWithLatestSlice { /** * Stream all the latest data files in the given partition. */ - Stream getLatestDataFiles(String partitionPath); + Stream getLatestBaseFiles(String partitionPath); /** * Get Latest data file for a partition and file-Id. */ - Option getLatestDataFile(String partitionPath, String fileId); + Option getLatestBaseFile(String partitionPath, String fileId); /** * Stream all the latest data files, in the file system view. */ - Stream getLatestDataFiles(); + Stream getLatestBaseFiles(); /** * Stream all the latest version data files in the given partition with precondition that commitTime(file) before * maxCommitTime. */ - Stream getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime); + Stream getLatestBaseFilesBeforeOrOn(String partitionPath, String maxCommitTime); /** * Stream all the latest data files pass. */ - Stream getLatestDataFilesInRange(List commitsToReturn); + Stream getLatestBaseFilesInRange(List commitsToReturn); } /** - * ReadOptimizedView - methods to provide a view of columnar data files only. + * Methods to provide a view of base files only. */ - interface ReadOptimizedView extends ReadOptimizedViewWithLatestSlice { + interface BaseFileOnlyView extends BaseFileOnlyViewWithLatestSlice { /** * Stream all the data file versions grouped by FileId for a given partition. */ - Stream getAllDataFiles(String partitionPath); + Stream getAllBaseFiles(String partitionPath); /** * Get the version of data file matching the instant time in the given partition. */ - Option getDataFileOn(String partitionPath, String instantTime, String fileId); + Option getBaseFileOn(String partitionPath, String instantTime, String fileId); } /** - * RealtimeView with methods to only access latest version of file-slice for the instant(s) passed. + * Methods to only access latest version of file-slice for the instant(s) passed. */ - interface RealtimeViewWithLatestSlice { + interface SliceViewWithLatestSlice { /** * Stream all the latest file slices in the given partition. @@ -131,9 +131,9 @@ public interface TableFileSystemView { } /** - * RealtimeView - methods to access a combination of columnar data files + log files with real time data. + * Methods to access a combination of base files + log file slices. */ - interface RealtimeView extends RealtimeViewWithLatestSlice { + interface SliceView extends SliceViewWithLatestSlice { /** * Stream all the file slices for a given partition, latest or not. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DataFileDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/BaseFileDTO.java similarity index 77% rename from hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DataFileDTO.java rename to hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/BaseFileDTO.java index fab7ad1df..408aafda9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DataFileDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/BaseFileDTO.java @@ -18,7 +18,7 @@ package org.apache.hudi.common.table.timeline.dto; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; @@ -27,7 +27,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; * The data transfer object of data file. */ @JsonIgnoreProperties(ignoreUnknown = true) -public class DataFileDTO { +public class BaseFileDTO { @JsonProperty("fileStatus") private FileStatusDTO fileStatus; @@ -36,27 +36,27 @@ public class DataFileDTO { @JsonProperty("fileLen") private long fileLen; - public static HoodieDataFile toHoodieDataFile(DataFileDTO dto) { + public static HoodieBaseFile toHoodieBaseFile(BaseFileDTO dto) { if (null == dto) { return null; } - HoodieDataFile dataFile = null; + HoodieBaseFile baseFile; if (null != dto.fileStatus) { - dataFile = new HoodieDataFile(FileStatusDTO.toFileStatus(dto.fileStatus)); + baseFile = new HoodieBaseFile(FileStatusDTO.toFileStatus(dto.fileStatus)); } else { - dataFile = new HoodieDataFile(dto.fullPath); - dataFile.setFileLen(dto.fileLen); + baseFile = new HoodieBaseFile(dto.fullPath); + baseFile.setFileLen(dto.fileLen); } - return dataFile; + return baseFile; } - public static DataFileDTO fromHoodieDataFile(HoodieDataFile dataFile) { + public static BaseFileDTO fromHoodieBaseFile(HoodieBaseFile dataFile) { if (null == dataFile) { return null; } - DataFileDTO dto = new DataFileDTO(); + BaseFileDTO dto = new BaseFileDTO(); dto.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus()); dto.fullPath = dataFile.getPath(); dto.fileLen = dataFile.getFileLen(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileSliceDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileSliceDTO.java index e10386971..b3f7d24b7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileSliceDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileSliceDTO.java @@ -32,8 +32,8 @@ import java.util.stream.Collectors; @JsonIgnoreProperties(ignoreUnknown = true) public class FileSliceDTO { - @JsonProperty("dataFile") - DataFileDTO dataFile; + @JsonProperty("baseFile") + BaseFileDTO baseFile; @JsonProperty("logFiles") List logFiles; @JsonProperty("partition") @@ -48,14 +48,14 @@ public class FileSliceDTO { dto.partitionPath = slice.getPartitionPath(); dto.baseInstantTime = slice.getBaseInstantTime(); dto.fileId = slice.getFileId(); - dto.dataFile = slice.getDataFile().map(DataFileDTO::fromHoodieDataFile).orElse(null); + dto.baseFile = slice.getBaseFile().map(BaseFileDTO::fromHoodieBaseFile).orElse(null); dto.logFiles = slice.getLogFiles().map(LogFileDTO::fromHoodieLogFile).collect(Collectors.toList()); return dto; } public static FileSlice toFileSlice(FileSliceDTO dto) { FileSlice slice = new FileSlice(dto.partitionPath, dto.baseInstantTime, dto.fileId); - slice.setDataFile(DataFileDTO.toHoodieDataFile(dto.dataFile)); + slice.setBaseFile(BaseFileDTO.toHoodieBaseFile(dto.baseFile)); dto.logFiles.stream().forEach(lf -> slice.addLogFile(LogFileDTO.toHoodieLogFile(lf))); return slice; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 3ffa9fbd8..ee28cdc49 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -20,7 +20,7 @@ package org.apache.hudi.common.table.view; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; @@ -133,16 +133,16 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV */ protected List buildFileGroups(FileStatus[] statuses, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) { - return buildFileGroups(convertFileStatusesToDataFiles(statuses), convertFileStatusesToLogFiles(statuses), timeline, + return buildFileGroups(convertFileStatusesToBaseFiles(statuses), convertFileStatusesToLogFiles(statuses), timeline, addPendingCompactionFileSlice); } - protected List buildFileGroups(Stream dataFileStream, + protected List buildFileGroups(Stream baseFileStream, Stream logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) { - Map, List> dataFiles = - dataFileStream.collect(Collectors.groupingBy((dataFile) -> { - String partitionPathStr = getPartitionPathFromFilePath(dataFile.getPath()); - return Pair.of(partitionPathStr, dataFile.getFileId()); + Map, List> baseFiles = + baseFileStream.collect(Collectors.groupingBy((baseFile) -> { + String partitionPathStr = getPartitionPathFromFilePath(baseFile.getPath()); + return Pair.of(partitionPathStr, baseFile.getFileId()); })); Map, List> logFiles = logFileStream.collect(Collectors.groupingBy((logFile) -> { @@ -151,15 +151,15 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV return Pair.of(partitionPathStr, logFile.getFileId()); })); - Set> fileIdSet = new HashSet<>(dataFiles.keySet()); + Set> fileIdSet = new HashSet<>(baseFiles.keySet()); fileIdSet.addAll(logFiles.keySet()); List fileGroups = new ArrayList<>(); fileIdSet.forEach(pair -> { String fileId = pair.getValue(); HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, timeline); - if (dataFiles.containsKey(pair)) { - dataFiles.get(pair).forEach(group::addDataFile); + if (baseFiles.containsKey(pair)) { + baseFiles.get(pair).forEach(group::addBaseFile); } if (logFiles.containsKey(pair)) { logFiles.get(pair).forEach(group::addLogFile); @@ -233,7 +233,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV storePartitionView(partitionPathStr, new ArrayList<>()); } } catch (IOException e) { - throw new HoodieIOException("Failed to list data files in partition " + partitionPathStr, e); + throw new HoodieIOException("Failed to list base files in partition " + partitionPathStr, e); } } else { LOG.debug("View already built for Partition :" + partitionPathStr + ", FOUND is "); @@ -245,14 +245,14 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV } /** - * Helper to convert file-status to data-files. + * Helper to convert file-status to base-files. * * @param statuses List of File-Status */ - private Stream convertFileStatusesToDataFiles(FileStatus[] statuses) { + private Stream convertFileStatusesToBaseFiles(FileStatus[] statuses) { Predicate roFilePredicate = fileStatus -> fileStatus.getPath().getName() - .contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()); - return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new); + .contains(metaClient.getTableConfig().getBaseFileFormat().getFileExtension()); + return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieBaseFile::new); } /** @@ -262,23 +262,23 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV */ private Stream convertFileStatusesToLogFiles(FileStatus[] statuses) { Predicate rtFilePredicate = fileStatus -> fileStatus.getPath().getName() - .contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension()); + .contains(metaClient.getTableConfig().getLogFileFormat().getFileExtension()); return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new); } /** - * With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore those - * data-files. + * With async compaction, it is possible to see partial/complete base-files due to inflight-compactions, Ignore those + * base-files. * - * @param dataFile Data File + * @param baseFile base File */ - protected boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) { - final String partitionPath = getPartitionPathFromFilePath(dataFile.getPath()); + protected boolean isBaseFileDueToPendingCompaction(HoodieBaseFile baseFile) { + final String partitionPath = getPartitionPathFromFilePath(baseFile.getPath()); Option> compactionWithInstantTime = - getPendingCompactionOperationWithInstant(new HoodieFileGroupId(partitionPath, dataFile.getFileId())); + getPendingCompactionOperationWithInstant(new HoodieFileGroupId(partitionPath, baseFile.getFileId())); return (compactionWithInstantTime.isPresent()) && (null != compactionWithInstantTime.get().getKey()) - && dataFile.getCommitTime().equals(compactionWithInstantTime.get().getKey()); + && baseFile.getCommitTime().equals(compactionWithInstantTime.get().getKey()); } /** @@ -296,15 +296,15 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV } /** - * With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore those - * data-files. + * With async compaction, it is possible to see partial/complete base-files due to inflight-compactions, Ignore those + * base-files. * * @param fileSlice File Slice */ - protected FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) { + protected FileSlice filterBaseFileAfterPendingCompaction(FileSlice fileSlice) { if (isFileSliceAfterPendingCompaction(fileSlice)) { LOG.info("File Slice (" + fileSlice + ") is in pending compaction"); - // Data file is filtered out of the file-slice as the corresponding compaction + // Base file is filtered out of the file-slice as the corresponding compaction // instant not completed yet. FileSlice transformed = new FileSlice(fileSlice.getPartitionPath(), fileSlice.getBaseInstantTime(), fileSlice.getFileId()); @@ -325,38 +325,38 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV } @Override - public final Stream getLatestDataFiles(String partitionStr) { + public final Stream getLatestBaseFiles(String partitionStr) { try { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestDataFiles(partitionPath); + return fetchLatestBaseFiles(partitionPath); } finally { readLock.unlock(); } } @Override - public final Stream getLatestDataFiles() { + public final Stream getLatestBaseFiles() { try { readLock.lock(); - return fetchLatestDataFiles(); + return fetchLatestBaseFiles(); } finally { readLock.unlock(); } } @Override - public final Stream getLatestDataFilesBeforeOrOn(String partitionStr, String maxCommitTime) { + public final Stream getLatestBaseFilesBeforeOrOn(String partitionStr, String maxCommitTime) { try { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); return fetchAllStoredFileGroups(partitionPath) - .map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllDataFiles() - .filter(dataFile -> HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), maxCommitTime, + .map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllBaseFiles() + .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), maxCommitTime, HoodieTimeline.LESSER_OR_EQUAL)) - .filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst())) + .filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst())) .filter(Option::isPresent).map(Option::get); } finally { readLock.unlock(); @@ -364,43 +364,43 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV } @Override - public final Option getDataFileOn(String partitionStr, String instantTime, String fileId) { + public final Option getBaseFileOn(String partitionStr, String instantTime, String fileId) { try { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllDataFiles() + return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles() .filter( - dataFile -> HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), instantTime, HoodieTimeline.EQUAL)) - .filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst().orElse(null)); + baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), instantTime, HoodieTimeline.EQUAL)) + .filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst().orElse(null)); } finally { readLock.unlock(); } } /** - * Get Latest data file for a partition and file-Id. + * Get Latest base file for a partition and file-Id. */ @Override - public final Option getLatestDataFile(String partitionStr, String fileId) { + public final Option getLatestBaseFile(String partitionStr, String fileId) { try { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestDataFile(partitionPath, fileId); + return fetchLatestBaseFile(partitionPath, fileId); } finally { readLock.unlock(); } } @Override - public final Stream getLatestDataFilesInRange(List commitsToReturn) { + public final Stream getLatestBaseFilesInRange(List commitsToReturn) { try { readLock.lock(); return fetchAllStoredFileGroups().map(fileGroup -> { return Option.fromJavaOptional( - fileGroup.getAllDataFiles().filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime()) - && !isDataFileDueToPendingCompaction(dataFile)).findFirst()); + fileGroup.getAllBaseFiles().filter(baseFile -> commitsToReturn.contains(baseFile.getCommitTime()) + && !isBaseFileDueToPendingCompaction(baseFile)).findFirst()); }).filter(Option::isPresent).map(Option::get); } finally { readLock.unlock(); @@ -408,14 +408,14 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV } @Override - public final Stream getAllDataFiles(String partitionStr) { + public final Stream getAllBaseFiles(String partitionStr) { try { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchAllDataFiles(partitionPath) + return fetchAllBaseFiles(partitionPath) .filter(df -> visibleCommitsAndCompactionTimeline.containsOrBeforeTimelineStarts(df.getCommitTime())) - .filter(df -> !isDataFileDueToPendingCompaction(df)); + .filter(df -> !isBaseFileDueToPendingCompaction(df)); } finally { readLock.unlock(); } @@ -427,7 +427,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestFileSlices(partitionPath).map(fs -> filterDataFileAfterPendingCompaction(fs)); + return fetchLatestFileSlices(partitionPath).map(this::filterBaseFileAfterPendingCompaction); } finally { readLock.unlock(); } @@ -443,7 +443,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); Option fs = fetchLatestFileSlice(partitionPath, fileId); - return fs.map(f -> filterDataFileAfterPendingCompaction(f)); + return fs.map(f -> filterBaseFileAfterPendingCompaction(f)); } finally { readLock.unlock(); } @@ -480,7 +480,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV ensurePartitionLoadedCorrectly(partitionPath); Stream fileSliceStream = fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime); if (includeFileSlicesInPendingCompaction) { - return fileSliceStream.map(fs -> filterDataFileAfterPendingCompaction(fs)); + return fileSliceStream.map(fs -> filterBaseFileAfterPendingCompaction(fs)); } else { return fileSliceStream.filter(fs -> !isPendingCompactionScheduledForFileId(fs.getFileGroupId())); } @@ -653,33 +653,33 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV } /** - * Default implementation for fetching latest data-files for the partition-path. + * Default implementation for fetching latest base-files for the partition-path. */ - Stream fetchLatestDataFiles(final String partitionPath) { - return fetchAllStoredFileGroups(partitionPath).map(this::getLatestDataFile).filter(Option::isPresent) + Stream fetchLatestBaseFiles(final String partitionPath) { + return fetchAllStoredFileGroups(partitionPath).map(this::getLatestBaseFile).filter(Option::isPresent) .map(Option::get); } - protected Option getLatestDataFile(HoodieFileGroup fileGroup) { + protected Option getLatestBaseFile(HoodieFileGroup fileGroup) { return Option - .fromJavaOptional(fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst()); + .fromJavaOptional(fileGroup.getAllBaseFiles().filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst()); } /** - * Default implementation for fetching latest data-files across all partitions. + * Default implementation for fetching latest base-files across all partitions. */ - Stream fetchLatestDataFiles() { - return fetchAllStoredFileGroups().map(this::getLatestDataFile).filter(Option::isPresent).map(Option::get); + Stream fetchLatestBaseFiles() { + return fetchAllStoredFileGroups().map(this::getLatestBaseFile).filter(Option::isPresent).map(Option::get); } /** - * Default implementation for fetching all data-files for a partition. + * Default implementation for fetching all base-files for a partition. * * @param partitionPath partition-path */ - Stream fetchAllDataFiles(String partitionPath) { - return fetchAllStoredFileGroups(partitionPath).map(HoodieFileGroup::getAllDataFiles) - .flatMap(dataFileList -> dataFileList); + Stream fetchAllBaseFiles(String partitionPath) { + return fetchAllStoredFileGroups(partitionPath).map(HoodieFileGroup::getAllBaseFiles) + .flatMap(baseFileList -> baseFileList); } /** @@ -719,8 +719,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, FileSlice penultimateSlice) { FileSlice merged = new FileSlice(penultimateSlice.getPartitionPath(), penultimateSlice.getBaseInstantTime(), penultimateSlice.getFileId()); - if (penultimateSlice.getDataFile().isPresent()) { - merged.setDataFile(penultimateSlice.getDataFile().get()); + if (penultimateSlice.getBaseFile().isPresent()) { + merged.setBaseFile(penultimateSlice.getBaseFile().get()); } // Add Log files from penultimate and last slices penultimateSlice.getLogFiles().forEach(merged::addLogFile); @@ -752,15 +752,15 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV } /** - * Default implementation for fetching latest data-file. + * Default implementation for fetching latest base-file. * * @param partitionPath Partition path * @param fileId File Id - * @return Data File if present + * @return base File if present */ - protected Option fetchLatestDataFile(String partitionPath, String fileId) { - return Option - .fromJavaOptional(fetchLatestDataFiles(partitionPath).filter(fs -> fs.getFileId().equals(fileId)).findFirst()); + protected Option fetchLatestBaseFile(String partitionPath, String fileId) { + return Option.fromJavaOptional(fetchLatestBaseFiles(partitionPath) + .filter(fs -> fs.getFileId().equals(fileId)).findFirst()); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java index 19209acf7..07b262d89 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java @@ -25,7 +25,7 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTimeline; @@ -318,13 +318,13 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl * Note that while finding the new data/log files added/removed, the path stored in metadata will be missing the * base-path,scheme and authority. Ensure the matching process takes care of this discrepancy. */ - Map viewDataFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices) - .map(FileSlice::getDataFile).filter(Option::isPresent).map(Option::get) + Map viewDataFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices) + .map(FileSlice::getBaseFile).filter(Option::isPresent).map(Option::get) .map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df)) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); // Note: Delta Log Files and Data FIles can be empty when adding/removing pending compactions - Map deltaDataFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices) - .map(FileSlice::getDataFile).filter(Option::isPresent).map(Option::get) + Map deltaDataFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices) + .map(FileSlice::getBaseFile).filter(Option::isPresent).map(Option::get) .map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df)) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java index e9087db95..5eb1173c0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java @@ -20,7 +20,7 @@ package org.apache.hudi.common.table.view; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.SyncableFileSystemView; @@ -120,39 +120,39 @@ public class PriorityBasedFileSystemView implements SyncableFileSystemView, Seri } @Override - public Stream getLatestDataFiles(String partitionPath) { - return execute(partitionPath, preferredView::getLatestDataFiles, secondaryView::getLatestDataFiles); + public Stream getLatestBaseFiles(String partitionPath) { + return execute(partitionPath, preferredView::getLatestBaseFiles, secondaryView::getLatestBaseFiles); } @Override - public Stream getLatestDataFiles() { - return execute(preferredView::getLatestDataFiles, secondaryView::getLatestDataFiles); + public Stream getLatestBaseFiles() { + return execute(preferredView::getLatestBaseFiles, secondaryView::getLatestBaseFiles); } @Override - public Stream getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) { - return execute(partitionPath, maxCommitTime, preferredView::getLatestDataFilesBeforeOrOn, - secondaryView::getLatestDataFilesBeforeOrOn); + public Stream getLatestBaseFilesBeforeOrOn(String partitionPath, String maxCommitTime) { + return execute(partitionPath, maxCommitTime, preferredView::getLatestBaseFilesBeforeOrOn, + secondaryView::getLatestBaseFilesBeforeOrOn); } @Override - public Option getLatestDataFile(String partitionPath, String fileId) { - return execute(partitionPath, fileId, preferredView::getLatestDataFile, secondaryView::getLatestDataFile); + public Option getLatestBaseFile(String partitionPath, String fileId) { + return execute(partitionPath, fileId, preferredView::getLatestBaseFile, secondaryView::getLatestBaseFile); } @Override - public Option getDataFileOn(String partitionPath, String instantTime, String fileId) { - return execute(partitionPath, instantTime, fileId, preferredView::getDataFileOn, secondaryView::getDataFileOn); + public Option getBaseFileOn(String partitionPath, String instantTime, String fileId) { + return execute(partitionPath, instantTime, fileId, preferredView::getBaseFileOn, secondaryView::getBaseFileOn); } @Override - public Stream getLatestDataFilesInRange(List commitsToReturn) { - return execute(commitsToReturn, preferredView::getLatestDataFilesInRange, secondaryView::getLatestDataFilesInRange); + public Stream getLatestBaseFilesInRange(List commitsToReturn) { + return execute(commitsToReturn, preferredView::getLatestBaseFilesInRange, secondaryView::getLatestBaseFilesInRange); } @Override - public Stream getAllDataFiles(String partitionPath) { - return execute(partitionPath, preferredView::getAllDataFiles, secondaryView::getAllDataFiles); + public Stream getAllBaseFiles(String partitionPath) { + return execute(partitionPath, preferredView::getAllBaseFiles, secondaryView::getAllBaseFiles); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index 62ef3e6a2..791417e52 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java @@ -20,14 +20,14 @@ package org.apache.hudi.common.table.view; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.SyncableFileSystemView; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO; -import org.apache.hudi.common.table.timeline.dto.DataFileDTO; +import org.apache.hudi.common.table.timeline.dto.BaseFileDTO; import org.apache.hudi.common.table.timeline.dto.FileGroupDTO; import org.apache.hudi.common.table.timeline.dto.FileSliceDTO; import org.apache.hudi.common.table.timeline.dto.InstantDTO; @@ -205,74 +205,74 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, } @Override - public Stream getLatestDataFiles(String partitionPath) { + public Stream getLatestBaseFiles(String partitionPath) { Map paramsMap = getParamsWithPartitionPath(partitionPath); try { - List dataFiles = executeRequest(LATEST_PARTITION_DATA_FILES_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); - return dataFiles.stream().map(DataFileDTO::toHoodieDataFile); + List dataFiles = executeRequest(LATEST_PARTITION_DATA_FILES_URL, paramsMap, + new TypeReference>() {}, RequestMethod.GET); + return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile); } catch (IOException e) { throw new HoodieRemoteException(e); } } @Override - public Stream getLatestDataFiles() { + public Stream getLatestBaseFiles() { Map paramsMap = getParams(); try { - List dataFiles = executeRequest(LATEST_ALL_DATA_FILES, paramsMap, - new TypeReference>() {}, RequestMethod.GET); - return dataFiles.stream().map(DataFileDTO::toHoodieDataFile); + List dataFiles = executeRequest(LATEST_ALL_DATA_FILES, paramsMap, + new TypeReference>() {}, RequestMethod.GET); + return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile); } catch (IOException e) { throw new HoodieRemoteException(e); } } @Override - public Stream getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) { + public Stream getLatestBaseFilesBeforeOrOn(String partitionPath, String maxCommitTime) { Map paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime); try { - List dataFiles = executeRequest(LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); - return dataFiles.stream().map(DataFileDTO::toHoodieDataFile); + List dataFiles = executeRequest(LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, paramsMap, + new TypeReference>() {}, RequestMethod.GET); + return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile); } catch (IOException e) { throw new HoodieRemoteException(e); } } @Override - public Option getDataFileOn(String partitionPath, String instantTime, String fileId) { + public Option getBaseFileOn(String partitionPath, String instantTime, String fileId) { Map paramsMap = getParamsWithAdditionalParams(partitionPath, new String[] {INSTANT_PARAM, FILEID_PARAM}, new String[] {instantTime, fileId}); try { - List dataFiles = executeRequest(LATEST_DATA_FILE_ON_INSTANT_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); - return Option.fromJavaOptional(dataFiles.stream().map(DataFileDTO::toHoodieDataFile).findFirst()); + List dataFiles = executeRequest(LATEST_DATA_FILE_ON_INSTANT_URL, paramsMap, + new TypeReference>() {}, RequestMethod.GET); + return Option.fromJavaOptional(dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile).findFirst()); } catch (IOException e) { throw new HoodieRemoteException(e); } } @Override - public Stream getLatestDataFilesInRange(List commitsToReturn) { + public Stream getLatestBaseFilesInRange(List commitsToReturn) { Map paramsMap = getParams(INSTANTS_PARAM, StringUtils.join(commitsToReturn.toArray(new String[0]), ",")); try { - List dataFiles = executeRequest(LATEST_DATA_FILES_RANGE_INSTANT_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); - return dataFiles.stream().map(DataFileDTO::toHoodieDataFile); + List dataFiles = executeRequest(LATEST_DATA_FILES_RANGE_INSTANT_URL, paramsMap, + new TypeReference>() {}, RequestMethod.GET); + return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile); } catch (IOException e) { throw new HoodieRemoteException(e); } } @Override - public Stream getAllDataFiles(String partitionPath) { + public Stream getAllBaseFiles(String partitionPath) { Map paramsMap = getParamsWithPartitionPath(partitionPath); try { - List dataFiles = - executeRequest(ALL_DATA_FILES, paramsMap, new TypeReference>() {}, RequestMethod.GET); - return dataFiles.stream().map(DataFileDTO::toHoodieDataFile); + List dataFiles = + executeRequest(ALL_DATA_FILES, paramsMap, new TypeReference>() {}, RequestMethod.GET); + return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile); } catch (IOException e) { throw new HoodieRemoteException(e); } @@ -439,12 +439,12 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, } @Override - public Option getLatestDataFile(String partitionPath, String fileId) { + public Option getLatestBaseFile(String partitionPath, String fileId) { Map paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId); try { - List dataFiles = executeRequest(LATEST_PARTITION_DATA_FILE_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); - return Option.fromJavaOptional(dataFiles.stream().map(DataFileDTO::toHoodieDataFile).findFirst()); + List dataFiles = executeRequest(LATEST_PARTITION_DATA_FILE_URL, paramsMap, + new TypeReference>() {}, RequestMethod.GET); + return Option.fromJavaOptional(dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile).findFirst()); } catch (IOException e) { throw new HoodieRemoteException(e); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java index 9ef99a63c..1a00bca98 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java @@ -20,7 +20,7 @@ package org.apache.hudi.common.table.view; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; @@ -174,7 +174,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste rocksDB.writeBatch(batch -> { fg.getAllFileSlicesIncludingInflight().forEach(fs -> { rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs); - fs.getDataFile().ifPresent(df -> { + fs.getBaseFile().ifPresent(df -> { rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs), df); }); @@ -218,8 +218,8 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste switch (mode) { case ADD: { FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime()); - oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df)); - fs.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df)); + oldSlice.getBaseFile().ifPresent(df -> newFileSlice.setBaseFile(df)); + fs.getBaseFile().ifPresent(df -> newFileSlice.setBaseFile(df)); Map newLogFiles = new HashMap<>(logFiles); deltaLogFiles.entrySet().stream().filter(e -> !logFiles.containsKey(e.getKey())) .forEach(p -> newLogFiles.put(p.getKey(), p.getValue())); @@ -230,15 +230,15 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste case REMOVE: { LOG.info("Removing old File Slice =" + fs); FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime()); - fs.getDataFile().orElseGet(() -> { - oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df)); + fs.getBaseFile().orElseGet(() -> { + oldSlice.getBaseFile().ifPresent(df -> newFileSlice.setBaseFile(df)); return null; }); deltaLogFiles.keySet().stream().forEach(p -> logFiles.remove(p)); // Add remaining log files back logFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf)); - if (newFileSlice.getDataFile().isPresent() || (newFileSlice.getLogFiles().count() > 0)) { + if (newFileSlice.getBaseFile().isPresent() || (newFileSlice.getLogFiles().count() > 0)) { LOG.info("Adding back new file-slice after remove FS=" + newFileSlice); return newFileSlice; } @@ -250,7 +250,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste } }).filter(Objects::nonNull).forEach(fs -> { rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs); - fs.getDataFile().ifPresent(df -> { + fs.getBaseFile().ifPresent(df -> { rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs), df); }); @@ -266,8 +266,8 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste } @Override - Stream fetchAllDataFiles(String partitionPath) { - return rocksDB.prefixSearch(schemaHelper.getColFamilyForView(), + Stream fetchAllBaseFiles(String partitionPath) { + return rocksDB.prefixSearch(schemaHelper.getColFamilyForView(), schemaHelper.getPrefixForDataFileViewByPartition(partitionPath)).map(Pair::getValue); } @@ -298,11 +298,11 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste } @Override - protected Option fetchLatestDataFile(String partitionPath, String fileId) { + protected Option fetchLatestBaseFile(String partitionPath, String fileId) { // Retries only file-slices of the file and filters for the latest return Option .ofNullable(rocksDB - .prefixSearch(schemaHelper.getColFamilyForView(), + .prefixSearch(schemaHelper.getColFamilyForView(), schemaHelper.getPrefixForDataFileViewByPartitionFile(partitionPath, fileId)) .map(Pair::getValue).reduce(null, (x, y) -> ((x == null) ? y diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java index 7d5f786a0..6ee79df5e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java @@ -69,8 +69,8 @@ public class CompactionUtils { builder.setFileId(fileSlice.getFileId()); builder.setBaseInstantTime(fileSlice.getBaseInstantTime()); builder.setDeltaFilePaths(fileSlice.getLogFiles().map(lf -> lf.getPath().getName()).collect(Collectors.toList())); - if (fileSlice.getDataFile().isPresent()) { - builder.setDataFilePath(fileSlice.getDataFile().get().getFileName()); + if (fileSlice.getBaseFile().isPresent()) { + builder.setDataFilePath(fileSlice.getBaseFile().get().getFileName()); } if (metricsCaptureFunction.isPresent()) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index 88284028f..fc05e2f01 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -22,14 +22,15 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.HoodieCommonTestHarness; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.SyncableFileSystemView; -import org.apache.hudi.common.table.TableFileSystemView; +import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView; +import org.apache.hudi.common.table.TableFileSystemView.SliceView; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; @@ -76,8 +77,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { private static String TEST_WRITE_TOKEN = "1-0-1"; protected SyncableFileSystemView fsView; - protected TableFileSystemView.ReadOptimizedView roView; - protected TableFileSystemView.RealtimeView rtView; + protected BaseFileOnlyView roView; + protected SliceView rtView; @Before public void init() throws IOException { @@ -135,13 +136,13 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { refreshFsView(); - List dataFiles = roView.getLatestDataFiles().collect(Collectors.toList()); + List dataFiles = roView.getLatestBaseFiles().collect(Collectors.toList()); assertTrue("No data file expected", dataFiles.isEmpty()); List fileSliceList = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList()); assertEquals(1, fileSliceList.size()); FileSlice fileSlice = fileSliceList.get(0); assertEquals("File-Id must be set correctly", fileId, fileSlice.getFileId()); - assertFalse("Data file for base instant must be present", fileSlice.getDataFile().isPresent()); + assertFalse("Data file for base instant must be present", fileSlice.getBaseFile().isPresent()); assertEquals("Base Instant for file-group set correctly", instantTime1, fileSlice.getBaseInstantTime()); List logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size()); @@ -154,7 +155,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals(1, fileSliceList.size()); fileSlice = fileSliceList.get(0); assertEquals("File-Id must be set correctly", fileId, fileSlice.getFileId()); - assertFalse("Data file for base instant must be present", fileSlice.getDataFile().isPresent()); + assertFalse("Data file for base instant must be present", fileSlice.getBaseFile().isPresent()); assertEquals("Base Instant for file-group set correctly", instantTime1, fileSlice.getBaseInstantTime()); logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size()); @@ -166,7 +167,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals(1, fileSliceList.size()); fileSlice = fileSliceList.get(0); assertEquals("File-Id must be set correctly", fileId, fileSlice.getFileId()); - assertFalse("Data file for base instant must be present", fileSlice.getDataFile().isPresent()); + assertFalse("Data file for base instant must be present", fileSlice.getBaseFile().isPresent()); assertEquals("Base Instant for file-group set correctly", instantTime1, fileSlice.getBaseInstantTime()); logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size()); @@ -176,7 +177,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Total number of file-slices in view matches expected", expNumTotalFileSlices, rtView.getAllFileSlices(partitionPath).count()); assertEquals("Total number of data-files in view matches expected", expNumTotalDataFiles, - roView.getAllDataFiles(partitionPath).count()); + roView.getAllBaseFiles(partitionPath).count()); assertEquals("Total number of file-groups in view matches expected", 1, fsView.getAllFileGroups(partitionPath).count()); } @@ -293,7 +294,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Expected latest file-slices", 1, slices.size()); assertEquals("Base-Instant must be compaction Instant", compactionRequestedTime, slices.get(0).getBaseInstantTime()); - assertFalse("Latest File Slice must not have data-file", slices.get(0).getDataFile().isPresent()); + assertFalse("Latest File Slice must not have data-file", slices.get(0).getBaseFile().isPresent()); assertEquals("Latest File Slice must not have any log-files", 0, slices.get(0).getLogFiles().count()); // Fake delta-ingestion after compaction-requested @@ -313,7 +314,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { saveAsComplete(commitTimeline, deltaInstant5, Option.empty()); refreshFsView(); - List dataFiles = roView.getAllDataFiles(partitionPath).collect(Collectors.toList()); + List dataFiles = roView.getAllBaseFiles(partitionPath).collect(Collectors.toList()); if (skipCreatingDataFile) { assertTrue("No data file expected", dataFiles.isEmpty()); } else { @@ -328,9 +329,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { FileSlice fileSlice = fileSliceList.get(0); assertEquals(fileId, fileSlice.getFileId()); if (!skipCreatingDataFile) { - assertEquals("Data file must be present", dataFileName, fileSlice.getDataFile().get().getFileName()); + assertEquals("Data file must be present", dataFileName, fileSlice.getBaseFile().get().getFileName()); } else { - assertFalse("No data-file expected as it was not created", fileSlice.getDataFile().isPresent()); + assertFalse("No data-file expected as it was not created", fileSlice.getBaseFile().isPresent()); } assertEquals("Base Instant of penultimate file-slice must be base instant", instantTime1, fileSlice.getBaseInstantTime()); @@ -346,7 +347,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Expect only one file-id", 1, fileSliceList.size()); fileSlice = fileSliceList.get(0); assertEquals(fileId, fileSlice.getFileId()); - assertFalse("No data-file expected in latest file-slice", fileSlice.getDataFile().isPresent()); + assertFalse("No data-file expected in latest file-slice", fileSlice.getBaseFile().isPresent()); assertEquals("Compaction requested instant must be base instant", compactionRequestedTime, fileSlice.getBaseInstantTime()); logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); @@ -355,28 +356,28 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName()); /** Data Files API tests */ - dataFiles = roView.getLatestDataFiles().collect(Collectors.toList()); + dataFiles = roView.getLatestBaseFiles().collect(Collectors.toList()); if (skipCreatingDataFile) { assertEquals("Expect no data file to be returned", 0, dataFiles.size()); } else { assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1)); } - dataFiles = roView.getLatestDataFiles(partitionPath).collect(Collectors.toList()); + dataFiles = roView.getLatestBaseFiles(partitionPath).collect(Collectors.toList()); if (skipCreatingDataFile) { assertEquals("Expect no data file to be returned", 0, dataFiles.size()); } else { assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1)); } - dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); + dataFiles = roView.getLatestBaseFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); if (skipCreatingDataFile) { assertEquals("Expect no data file to be returned", 0, dataFiles.size()); } else { assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); dataFiles.forEach(df -> assertEquals("Expect data-file for instant 1 be returned", df.getCommitTime(), instantTime1)); } - dataFiles = roView.getLatestDataFilesInRange(allInstantTimes).collect(Collectors.toList()); + dataFiles = roView.getLatestBaseFilesInRange(allInstantTimes).collect(Collectors.toList()); if (skipCreatingDataFile) { assertEquals("Expect no data file to be returned", 0, dataFiles.size()); } else { @@ -415,8 +416,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { List allRawFileSlices = getAllRawFileSlices(partitionPath).collect(Collectors.toList()); dataFiles = allRawFileSlices.stream().flatMap(slice -> { - if (slice.getDataFile().isPresent()) { - return Stream.of(slice.getDataFile().get()); + if (slice.getBaseFile().isPresent()) { + return Stream.of(slice.getBaseFile().get()); } return Stream.empty(); }).collect(Collectors.toList()); @@ -424,7 +425,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { if (includeInvalidAndInflight) { assertEquals("Inflight/Orphan data-file is also expected", 2 + (isCompactionInFlight ? 1 : 0) + (skipCreatingDataFile ? 0 : 1), dataFiles.size()); - Set fileNames = dataFiles.stream().map(HoodieDataFile::getFileName).collect(Collectors.toSet()); + Set fileNames = dataFiles.stream().map(HoodieBaseFile::getFileName).collect(Collectors.toSet()); assertTrue("Expect orphan data-file to be present", fileNames.contains(orphanDataFileName)); assertTrue("Expect inflight data-file to be present", fileNames.contains(inflightDataFileName)); if (!skipCreatingDataFile) { @@ -448,26 +449,26 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Orphan File Slice with data-file check base-commit", invalidInstantId, orphanFileSliceWithDataFile.getBaseInstantTime()); assertEquals("Orphan File Slice with data-file check data-file", orphanDataFileName, - orphanFileSliceWithDataFile.getDataFile().get().getFileName()); + orphanFileSliceWithDataFile.getBaseFile().get().getFileName()); assertEquals("Orphan File Slice with data-file check data-file", 0, orphanFileSliceWithDataFile.getLogFiles().count()); assertEquals("Inflight File Slice with data-file check base-commit", inflightDeltaInstantTime, inflightFileSliceWithDataFile.getBaseInstantTime()); assertEquals("Inflight File Slice with data-file check data-file", inflightDataFileName, - inflightFileSliceWithDataFile.getDataFile().get().getFileName()); + inflightFileSliceWithDataFile.getBaseFile().get().getFileName()); assertEquals("Inflight File Slice with data-file check data-file", 0, inflightFileSliceWithDataFile.getLogFiles().count()); assertEquals("Orphan File Slice with log-file check base-commit", invalidInstantId, orphanFileSliceWithLogFile.getBaseInstantTime()); assertFalse("Orphan File Slice with log-file check data-file", - orphanFileSliceWithLogFile.getDataFile().isPresent()); + orphanFileSliceWithLogFile.getBaseFile().isPresent()); logFiles = orphanFileSliceWithLogFile.getLogFiles().collect(Collectors.toList()); assertEquals("Orphan File Slice with log-file check data-file", 1, logFiles.size()); assertEquals("Orphan File Slice with log-file check data-file", orphanLogFileName, logFiles.get(0).getFileName()); assertEquals("Inflight File Slice with log-file check base-commit", inflightDeltaInstantTime, inflightFileSliceWithLogFile.getBaseInstantTime()); assertFalse("Inflight File Slice with log-file check data-file", - inflightFileSliceWithLogFile.getDataFile().isPresent()); + inflightFileSliceWithLogFile.getBaseFile().isPresent()); logFiles = inflightFileSliceWithLogFile.getLogFiles().collect(Collectors.toList()); assertEquals("Inflight File Slice with log-file check data-file", 1, logFiles.size()); assertEquals("Inflight File Slice with log-file check data-file", inflightLogFileName, @@ -486,11 +487,11 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { commitTimeline.saveAsComplete(compactionInstant, Option.empty()); refreshFsView(); // populate the cache - roView.getAllDataFiles(partitionPath); + roView.getAllBaseFiles(partitionPath); fileSliceList = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList()); LOG.info("FILESLICE LIST=" + fileSliceList); - dataFiles = fileSliceList.stream().map(FileSlice::getDataFile).filter(Option::isPresent).map(Option::get) + dataFiles = fileSliceList.stream().map(FileSlice::getBaseFile).filter(Option::isPresent).map(Option::get) .collect(Collectors.toList()); assertEquals("Expect only one data-files in latest view as there is only one file-group", 1, dataFiles.size()); assertEquals("Data Filename must match", compactDataFileName, dataFiles.get(0).getFileName()); @@ -498,7 +499,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { fileSlice = fileSliceList.get(0); assertEquals("Check file-Id is set correctly", fileId, fileSlice.getFileId()); assertEquals("Check data-filename is set correctly", compactDataFileName, - fileSlice.getDataFile().get().getFileName()); + fileSlice.getBaseFile().get().getFileName()); assertEquals("Ensure base-instant is now compaction request instant", compactionRequestedTime, fileSlice.getBaseInstantTime()); logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); @@ -507,23 +508,23 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName()); /** Data Files API tests */ - dataFiles = roView.getLatestDataFiles().collect(Collectors.toList()); + dataFiles = roView.getLatestBaseFiles().collect(Collectors.toList()); assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime)); - dataFiles = roView.getLatestDataFiles(partitionPath).collect(Collectors.toList()); + dataFiles = roView.getLatestBaseFiles(partitionPath).collect(Collectors.toList()); assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime)); - dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); + dataFiles = roView.getLatestBaseFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime)); - dataFiles = roView.getLatestDataFilesInRange(allInstantTimes).collect(Collectors.toList()); + dataFiles = roView.getLatestBaseFilesInRange(allInstantTimes).collect(Collectors.toList()); assertEquals("Expect only one data-file to be sent", 1, dataFiles.size()); dataFiles.forEach(df -> assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(), compactionRequestedTime)); assertEquals("Total number of file-slices in partitions matches expected", expTotalFileSlices, rtView.getAllFileSlices(partitionPath).count()); assertEquals("Total number of data-files in partitions matches expected", expTotalDataFiles, - roView.getAllDataFiles(partitionPath).count()); + roView.getAllBaseFiles(partitionPath).count()); // file-groups includes inflight/invalid file-ids assertEquals("Total number of file-groups in partitions matches expected", 5, fsView.getAllFileGroups(partitionPath).count()); @@ -535,7 +536,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { new File(basePath + "/" + partitionPath).mkdirs(); String fileId = UUID.randomUUID().toString(); - assertFalse("No commit, should not find any data file", roView.getLatestDataFiles(partitionPath) + assertFalse("No commit, should not find any data file", roView.getLatestBaseFiles(partitionPath) .anyMatch(dfile -> dfile.getFileId().equals(fileId))); // Only one commit, but is not safe @@ -543,7 +544,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId); new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); refreshFsView(); - assertFalse("No commit, should not find any data file", roView.getLatestDataFiles(partitionPath) + assertFalse("No commit, should not find any data file", roView.getLatestBaseFiles(partitionPath) .anyMatch(dfile -> dfile.getFileId().equals(fileId))); // Make this commit safe @@ -551,7 +552,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); saveAsComplete(commitTimeline, instant1, Option.empty()); refreshFsView(); - assertEquals("", fileName1, roView.getLatestDataFiles(partitionPath) + assertEquals("", fileName1, roView.getLatestBaseFiles(partitionPath) .filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName()); // Do another commit, but not safe @@ -559,14 +560,14 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { String fileName2 = FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId); new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); refreshFsView(); - assertEquals("", fileName1, roView.getLatestDataFiles(partitionPath) + assertEquals("", fileName1, roView.getLatestBaseFiles(partitionPath) .filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName()); // Make it safe HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2); saveAsComplete(commitTimeline, instant2, Option.empty()); refreshFsView(); - assertEquals("", fileName2, roView.getLatestDataFiles(partitionPath) + assertEquals("", fileName2, roView.getLatestBaseFiles(partitionPath) .filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName()); } @@ -648,11 +649,11 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals(isLatestFileSliceOnly ? 1 : 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.getLatestBaseFilesBeforeOrOn("2016/05/01", commitTime4).collect(Collectors.toList()); assertEquals(3, dataFileList.size()); Set filenames = Sets.newHashSet(); - for (HoodieDataFile status : dataFileList) { + for (HoodieBaseFile status : dataFileList) { filenames.add(status.getFileName()); } assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))); @@ -676,10 +677,10 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { .contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN))); // Reset the max commit time - List dataFiles = - roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3).collect(Collectors.toList()); + List dataFiles = + roView.getLatestBaseFilesBeforeOrOn("2016/05/01", commitTime3).collect(Collectors.toList()); filenames = Sets.newHashSet(); - for (HoodieDataFile status : dataFiles) { + for (HoodieBaseFile status : dataFiles) { filenames.add(status.getFileName()); } if (!isLatestFileSliceOnly) { @@ -739,7 +740,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { for (HoodieFileGroup fileGroup : fileGroups) { String fileId = fileGroup.getFileGroupId().getFileId(); Set filenames = Sets.newHashSet(); - fileGroup.getAllDataFiles().forEach(dataFile -> { + fileGroup.getAllBaseFiles().forEach(dataFile -> { assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId()); filenames.add(dataFile.getFileName()); }); @@ -811,13 +812,13 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { refreshFsView(); // Populate view for partition - roView.getAllDataFiles("2016/05/01/"); + roView.getAllBaseFiles("2016/05/01/"); - List dataFiles = - roView.getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3)).collect(Collectors.toList()); + List dataFiles = + roView.getLatestBaseFilesInRange(Lists.newArrayList(commitTime2, commitTime3)).collect(Collectors.toList()); assertEquals(isLatestFileSliceOnly ? 2 : 3, dataFiles.size()); Set filenames = Sets.newHashSet(); - for (HoodieDataFile status : dataFiles) { + for (HoodieBaseFile status : dataFiles) { filenames.add(status.getFileName()); } @@ -833,15 +834,15 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { for (FileSlice slice : slices) { if (slice.getFileId().equals(fileId1)) { assertEquals(slice.getBaseInstantTime(), commitTime3); - assertTrue(slice.getDataFile().isPresent()); + assertTrue(slice.getBaseFile().isPresent()); assertEquals(slice.getLogFiles().count(), 0); } else if (slice.getFileId().equals(fileId2)) { assertEquals(slice.getBaseInstantTime(), commitTime3); - assertTrue(slice.getDataFile().isPresent()); + assertTrue(slice.getBaseFile().isPresent()); assertEquals(slice.getLogFiles().count(), 1); } else if (slice.getFileId().equals(fileId3)) { assertEquals(slice.getBaseInstantTime(), commitTime4); - assertTrue(slice.getDataFile().isPresent()); + assertTrue(slice.getBaseFile().isPresent()); assertEquals(slice.getLogFiles().count(), 0); } } @@ -883,12 +884,12 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals(7, statuses.length); refreshFsView(); - List dataFiles = - roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2).collect(Collectors.toList()); + List dataFiles = + roView.getLatestBaseFilesBeforeOrOn(partitionPath, commitTime2).collect(Collectors.toList()); if (!isLatestFileSliceOnly) { assertEquals(2, dataFiles.size()); Set filenames = Sets.newHashSet(); - for (HoodieDataFile status : dataFiles) { + for (HoodieBaseFile status : dataFiles) { filenames.add(status.getFileName()); } assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))); @@ -952,7 +953,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals(10, statuses.length); refreshFsView(); - fsView.getAllDataFiles(partitionPath); + fsView.getAllBaseFiles(partitionPath); List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); assertEquals(3, fileGroups.size()); for (HoodieFileGroup fileGroup : fileGroups) { @@ -980,10 +981,10 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { } } - List statuses1 = roView.getLatestDataFiles().collect(Collectors.toList()); + List statuses1 = roView.getLatestBaseFiles().collect(Collectors.toList()); assertEquals(3, statuses1.size()); Set filenames = Sets.newHashSet(); - for (HoodieDataFile status : statuses1) { + for (HoodieBaseFile status : statuses1) { filenames.add(status.getFileName()); } assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))); @@ -1092,10 +1093,10 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { refreshFsView(); // Test Data Files - List dataFiles = roView.getAllDataFiles(partitionPath1).collect(Collectors.toList()); + List dataFiles = roView.getAllBaseFiles(partitionPath1).collect(Collectors.toList()); assertEquals("One data-file is expected as there is only one file-group", 1, dataFiles.size()); assertEquals("Expect only valid commit", "1", dataFiles.get(0).getCommitTime()); - dataFiles = roView.getAllDataFiles(partitionPath2).collect(Collectors.toList()); + dataFiles = roView.getAllBaseFiles(partitionPath2).collect(Collectors.toList()); assertEquals("One data-file is expected as there is only one file-group", 1, dataFiles.size()); assertEquals("Expect only valid commit", "1", dataFiles.get(0).getCommitTime()); @@ -1106,7 +1107,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Expect file-slice to be merged", 1, fileSliceList.size()); FileSlice fileSlice = fileSliceList.get(0); assertEquals(fileId, fileSlice.getFileId()); - assertEquals("Data file must be present", dataFileName, fileSlice.getDataFile().get().getFileName()); + assertEquals("Data file must be present", dataFileName, fileSlice.getBaseFile().get().getFileName()); assertEquals("Base Instant of penultimate file-slice must be base instant", instantTime1, fileSlice.getBaseInstantTime()); List logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); @@ -1120,7 +1121,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { assertEquals("Expect only one file-id", 1, fileSliceList.size()); fileSlice = fileSliceList.get(0); assertEquals(fileId, fileSlice.getFileId()); - assertFalse("No data-file expected in latest file-slice", fileSlice.getDataFile().isPresent()); + assertFalse("No data-file expected in latest file-slice", fileSlice.getBaseFile().isPresent()); assertEquals("Compaction requested instant must be base instant", compactionRequestedTime, fileSlice.getBaseInstantTime()); logFiles = fileSlice.getLogFiles().collect(Collectors.toList()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java index 31ff91008..7a4efccb6 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java @@ -29,7 +29,7 @@ import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieTableType; @@ -502,7 +502,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { view.getLatestFileSlices(p).forEach(fs -> { Assert.assertEquals(instantTime, fs.getBaseInstantTime()); Assert.assertEquals(p, fs.getPartitionPath()); - Assert.assertFalse(fs.getDataFile().isPresent()); + Assert.assertFalse(fs.getBaseFile().isPresent()); }); view.getLatestMergedFileSlicesBeforeOrOn(p, instantTime).forEach(fs -> { Assert @@ -625,7 +625,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { }); } else { partitions.forEach(p -> { - view.getLatestDataFiles(p).forEach(f -> { + view.getLatestBaseFiles(p).forEach(f -> { Assert.assertEquals(instant, f.getCommitTime()); }); }); @@ -676,10 +676,10 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { FileSlice slice2 = e2.getValue(); Assert.assertEquals(slice1.getBaseInstantTime(), slice2.getBaseInstantTime()); Assert.assertEquals(slice1.getFileId(), slice2.getFileId()); - Assert.assertEquals(slice1.getDataFile().isPresent(), slice2.getDataFile().isPresent()); - if (slice1.getDataFile().isPresent()) { - HoodieDataFile df1 = slice1.getDataFile().get(); - HoodieDataFile df2 = slice2.getDataFile().get(); + Assert.assertEquals(slice1.getBaseFile().isPresent(), slice2.getBaseFile().isPresent()); + if (slice1.getBaseFile().isPresent()) { + HoodieBaseFile df1 = slice1.getBaseFile().get(); + HoodieBaseFile df2 = slice2.getBaseFile().get(); Assert.assertEquals(df1.getCommitTime(), df2.getCommitTime()); Assert.assertEquals(df1.getFileId(), df2.getFileId()); Assert.assertEquals(df1.getFileName(), df2.getFileName()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/CompactionTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/CompactionTestUtils.java index 65b11c4c3..46783ef80 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/CompactionTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/CompactionTestUtils.java @@ -21,7 +21,7 @@ package org.apache.hudi.common.util; import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTestUtils; @@ -157,7 +157,7 @@ public class CompactionTestUtils { instantId, fileId, Option.of(2)); FileSlice slice = new FileSlice(DEFAULT_PARTITION_PATHS[0], instantId, fileId); if (createDataFile) { - slice.setDataFile(new TestHoodieDataFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0] + "/" + slice.setBaseFile(new TestHoodieBaseFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0] + "/" + FSUtils.makeDataFileName(instantId, TEST_WRITE_TOKEN, fileId))); } String logFilePath1 = HoodieTestUtils.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], @@ -186,11 +186,11 @@ public class CompactionTestUtils { /** * The hoodie data file for testing. */ - public static class TestHoodieDataFile extends HoodieDataFile { + public static class TestHoodieBaseFile extends HoodieBaseFile { private final String path; - public TestHoodieDataFile(String path) { + public TestHoodieBaseFile(String path) { super(path); this.path = path; } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java index d85607842..94d8b66fa 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java @@ -22,12 +22,12 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.HoodieCommonTestHarness; import org.apache.hudi.common.model.FileSlice; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.util.CompactionTestUtils.TestHoodieDataFile; +import org.apache.hudi.common.util.CompactionTestUtils.TestHoodieBaseFile; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.versioning.compaction.CompactionPlanMigrator; @@ -97,7 +97,7 @@ public class TestCompactionUtils extends HoodieCommonTestHarness { // File Slice with data-file but no log files FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noLog1"); - noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet")); + noLogFileSlice.setBaseFile(new TestHoodieBaseFile("/tmp/noLog_1_000.parquet")); op = CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Option.of(metricsCaptureFn)); testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0], LATEST_COMPACTION_METADATA_VERSION); @@ -113,7 +113,7 @@ public class TestCompactionUtils extends HoodieCommonTestHarness { // File Slice with data-file and log files present FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1"); - fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet")); + fileSlice.setBaseFile(new TestHoodieBaseFile("/tmp/noLog_1_000.parquet")); fileSlice.addLogFile( new HoodieLogFile(new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))); fileSlice.addLogFile( @@ -129,13 +129,13 @@ public class TestCompactionUtils extends HoodieCommonTestHarness { Path fullPartitionPath = new Path(new Path(metaClient.getBasePath()), DEFAULT_PARTITION_PATHS[0]); FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "empty1"); FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1"); - fileSlice.setDataFile(new TestHoodieDataFile(fullPartitionPath.toString() + "/data1_1_000.parquet")); + fileSlice.setBaseFile(new TestHoodieBaseFile(fullPartitionPath.toString() + "/data1_1_000.parquet")); fileSlice.addLogFile(new HoodieLogFile( new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))))); fileSlice.addLogFile(new HoodieLogFile( new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))))); FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noLog1"); - noLogFileSlice.setDataFile(new TestHoodieDataFile(fullPartitionPath.toString() + "/noLog_1_000.parquet")); + noLogFileSlice.setBaseFile(new TestHoodieBaseFile(fullPartitionPath.toString() + "/noLog_1_000.parquet")); FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1"); noDataFileSlice.addLogFile(new HoodieLogFile( new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))))); @@ -241,8 +241,8 @@ public class TestCompactionUtils extends HoodieCommonTestHarness { Assert.assertEquals("Partition path is correct", expPartitionPath, op.getPartitionPath()); Assert.assertEquals("Same base-instant", slice.getBaseInstantTime(), op.getBaseInstantTime()); Assert.assertEquals("Same file-id", slice.getFileId(), op.getFileId()); - if (slice.getDataFile().isPresent()) { - HoodieDataFile df = slice.getDataFile().get(); + if (slice.getBaseFile().isPresent()) { + HoodieBaseFile df = slice.getBaseFile().get(); Assert.assertEquals("Same data-file", version == COMPACTION_METADATA_VERSION_1 ? df.getPath() : df.getFileName(), op.getDataFilePath()); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHiveUtil.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHiveUtil.java index 17fbf4eec..ba8e04213 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHiveUtil.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHiveUtil.java @@ -37,8 +37,8 @@ public class HoodieHiveUtil { public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp"; public static final String HOODIE_MAX_COMMIT_PATTERN = "hoodie.%s.consume.max.commits"; public static final String INCREMENTAL_SCAN_MODE = "INCREMENTAL"; - public static final String LATEST_SCAN_MODE = "LATEST"; - public static final String DEFAULT_SCAN_MODE = LATEST_SCAN_MODE; + public static final String SNAPSHOT_SCAN_MODE = "SNAPSHOT"; + public static final String DEFAULT_SCAN_MODE = SNAPSHOT_SCAN_MODE; public static final int DEFAULT_MAX_COMMITS = 1; public static final int MAX_COMMIT_ALL = -1; public static final int DEFAULT_LEVELS_TO_BASEPATH = 3; @@ -60,13 +60,6 @@ public class HoodieHiveUtil { return job.getConfiguration().get(startCommitTimestampName); } - public static String readMode(JobContext job, String tableName) { - String modePropertyName = String.format(HOODIE_CONSUME_MODE_PATTERN, tableName); - String mode = job.getConfiguration().get(modePropertyName, DEFAULT_SCAN_MODE); - LOG.info(modePropertyName + ": " + mode); - return mode; - } - public static Path getNthParent(Path path, int n) { Path parent = path; for (int i = 0; i < n; i++) { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java index 3d91f5da1..e7eb99014 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java @@ -41,11 +41,11 @@ import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapreduce.Job; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; -import org.apache.hudi.common.table.TableFileSystemView; +import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.exception.HoodieIOException; @@ -177,12 +177,12 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement // Mutate the JobConf to set the input paths to only partitions touched by incremental pull. setInputPaths(job, incrementalInputPaths); FileStatus[] fileStatuses = super.listStatus(job); - TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(tableMetaClient, timeline, + BaseFileOnlyView roView = new HoodieTableFileSystemView(tableMetaClient, timeline, fileStatuses); List commitsList = commitsToCheck.stream().map(s -> s.getTimestamp()).collect(Collectors.toList()); - List filteredFiles = roView.getLatestDataFilesInRange(commitsList).collect(Collectors.toList()); + List filteredFiles = roView.getLatestBaseFilesInRange(commitsList).collect(Collectors.toList()); List returns = new ArrayList<>(); - for (HoodieDataFile filteredFile : filteredFiles) { + for (HoodieBaseFile filteredFile : filteredFiles) { LOG.debug("Processing incremental hoodie file - " + filteredFile.getPath()); filteredFile = checkFileStatus(filteredFile); returns.add(filteredFile.getFileStatus()); @@ -238,12 +238,12 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement } // 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, timeline, statuses); + BaseFileOnlyView roView = new HoodieTableFileSystemView(metadata, timeline, statuses); // filter files on the latest commit found - List filteredFiles = roView.getLatestDataFiles().collect(Collectors.toList()); + List filteredFiles = roView.getLatestBaseFiles().collect(Collectors.toList()); LOG.info("Total paths to process after hoodie filter " + filteredFiles.size()); List returns = new ArrayList<>(); - for (HoodieDataFile filteredFile : filteredFiles) { + for (HoodieBaseFile filteredFile : filteredFiles) { if (LOG.isDebugEnabled()) { LOG.debug("Processing latest hoodie file - " + filteredFile.getPath()); } @@ -258,13 +258,13 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement * super.listStatus() and gets back a FileStatus[] 2. Then it creates the HoodieTableMetaClient for the paths listed. * 3. Generation of splits looks at FileStatus size to create splits, which skips this file */ - private HoodieDataFile checkFileStatus(HoodieDataFile dataFile) throws IOException { + private HoodieBaseFile checkFileStatus(HoodieBaseFile dataFile) throws IOException { Path dataPath = dataFile.getFileStatus().getPath(); try { if (dataFile.getFileSize() == 0) { FileSystem fs = dataPath.getFileSystem(conf); LOG.info("Refreshing file status " + dataFile.getPath()); - return new HoodieDataFile(fs.getFileStatus(dataPath)); + return new HoodieBaseFile(fs.getFileStatus(dataPath)); } return dataFile; } catch (IOException e) { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java index bad365713..66ec864c4 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java @@ -18,7 +18,7 @@ package org.apache.hudi.hadoop; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; @@ -139,14 +139,14 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), baseDir.toString()); HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), fs.listStatus(folder)); - List latestFiles = fsView.getLatestDataFiles().collect(Collectors.toList()); + List latestFiles = fsView.getLatestBaseFiles().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); - for (HoodieDataFile lfile : latestFiles) { + for (HoodieBaseFile lfile : latestFiles) { hoodiePathCache.get(folder.toString()).add(new Path(lfile.getPath())); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java index d7b50d4f1..1bfc446e0 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -336,7 +336,7 @@ public abstract class AbstractRealtimeRecordReader { */ private void init() throws IOException { Schema schemaFromLogFile = - LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaFilePaths(), jobConf); + LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaLogPaths(), jobConf); if (schemaFromLogFile == null) { writerSchema = new AvroSchemaConverter().convert(baseFileSchema); LOG.debug("Writer Schema From Parquet => " + writerSchema.getFields()); @@ -360,7 +360,7 @@ public abstract class AbstractRealtimeRecordReader { readerSchema = generateProjectionSchema(writerSchema, schemaFieldsMap, projectionFields); LOG.info(String.format("About to read compacted logs %s for base split %s, projecting cols %s", - split.getDeltaFilePaths(), split.getPath(), projectionFields)); + split.getDeltaLogPaths(), split.getPath(), projectionFields)); } private Schema constructHiveOrderedSchema(Schema writerSchema, Map schemaFieldsMap) { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java index c955bf64b..908ec8778 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java @@ -32,7 +32,7 @@ import java.util.List; */ public class HoodieRealtimeFileSplit extends FileSplit { - private List deltaFilePaths; + private List deltaLogPaths; private String maxCommitTime; @@ -42,16 +42,16 @@ public class HoodieRealtimeFileSplit extends FileSplit { super(); } - public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List deltaLogFiles, String maxCommitTime) + public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List deltaLogPaths, String maxCommitTime) throws IOException { super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(), baseSplit.getLocations()); - this.deltaFilePaths = deltaLogFiles; + this.deltaLogPaths = deltaLogPaths; this.maxCommitTime = maxCommitTime; this.basePath = basePath; } - public List getDeltaFilePaths() { - return deltaFilePaths; + public List getDeltaLogPaths() { + return deltaLogPaths; } public String getMaxCommitTime() { @@ -79,8 +79,8 @@ public class HoodieRealtimeFileSplit extends FileSplit { super.write(out); writeString(basePath, out); writeString(maxCommitTime, out); - out.writeInt(deltaFilePaths.size()); - for (String logFilePath : deltaFilePaths) { + out.writeInt(deltaLogPaths.size()); + for (String logFilePath : deltaLogPaths) { writeString(logFilePath, out); } } @@ -91,15 +91,15 @@ public class HoodieRealtimeFileSplit extends FileSplit { basePath = readString(in); maxCommitTime = readString(in); int totalLogFiles = in.readInt(); - deltaFilePaths = new ArrayList<>(totalLogFiles); + deltaLogPaths = new ArrayList<>(totalLogFiles); for (int i = 0; i < totalLogFiles; i++) { - deltaFilePaths.add(readString(in)); + deltaLogPaths.add(readString(in)); } } @Override public String toString() { - return "HoodieRealtimeFileSplit{DataPath=" + getPath() + ", deltaFilePaths=" + deltaFilePaths + return "HoodieRealtimeFileSplit{DataPath=" + getPath() + ", deltaLogPaths=" + deltaLogPaths + ", maxCommitTime='" + maxCommitTime + '\'' + ", basePath='" + basePath + '\'' + '}'; } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java index deeaaf481..45e88376b 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -61,7 +61,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader // but can return records for completed commits > the commit we are trying to read (if using // readCommit() API) return new HoodieMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(), - split.getDeltaFilePaths(), usesCustomPayload ? getWriterSchema() : getReaderSchema(), split.getMaxCommitTime(), + split.getDeltaLogPaths(), usesCustomPayload ? getWriterSchema() : getReaderSchema(), split.getMaxCommitTime(), getMaxCompactionMemoryInBytes(), Boolean .valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java index f1a8eb87e..96172bf1b 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java @@ -77,7 +77,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader // Consumer of this record reader this.iterator = this.executor.getQueue().iterator(); this.logRecordScanner = new HoodieUnMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf), - split.getBasePath(), split.getDeltaFilePaths(), getReaderSchema(), split.getMaxCommitTime(), + split.getBasePath(), split.getDeltaLogPaths(), getReaderSchema(), split.getMaxCommitTime(), Boolean .valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), record -> { diff --git a/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java b/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java index f21335ae3..37f446337 100644 --- a/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java +++ b/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java @@ -68,6 +68,9 @@ public class HiveSyncConfig implements Serializable { @Parameter(names = {"--use-jdbc"}, description = "Hive jdbc connect url") public Boolean useJdbc = true; + @Parameter(names = {"--skip-ro-suffix"}, description = "Skip the `_ro` suffix for Read optimized table, when registering") + public Boolean skipROSuffix = false; + @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; diff --git a/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncTool.java index e20f5c9c1..8684f3bd9 100644 --- a/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncTool.java +++ b/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncTool.java @@ -53,30 +53,44 @@ import java.util.stream.Collectors; public class HiveSyncTool { private static final Logger LOG = LogManager.getLogger(HiveSyncTool.class); - private final HoodieHiveClient hoodieHiveClient; - public static final String SUFFIX_REALTIME_TABLE = "_rt"; + public static final String SUFFIX_SNAPSHOT_TABLE = "_rt"; + public static final String SUFFIX_READ_OPTIMIZED_TABLE = "_ro"; + private final HiveSyncConfig cfg; + private final HoodieHiveClient hoodieHiveClient; + private final String snapshotTableName; + private final Option roTableTableName; public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { this.hoodieHiveClient = new HoodieHiveClient(cfg, configuration, fs); this.cfg = cfg; + switch (hoodieHiveClient.getTableType()) { + case COPY_ON_WRITE: + this.snapshotTableName = cfg.tableName; + this.roTableTableName = Option.empty(); + break; + case MERGE_ON_READ: + this.snapshotTableName = cfg.tableName + SUFFIX_SNAPSHOT_TABLE; + this.roTableTableName = cfg.skipROSuffix ? Option.of(cfg.tableName) : + Option.of(cfg.tableName + SUFFIX_READ_OPTIMIZED_TABLE); + break; + default: + LOG.error("Unknown table type " + hoodieHiveClient.getTableType()); + throw new InvalidTableException(hoodieHiveClient.getBasePath()); + } } public void syncHoodieTable() throws ClassNotFoundException { try { switch (hoodieHiveClient.getTableType()) { case COPY_ON_WRITE: - syncHoodieTable(false); + syncHoodieTable(snapshotTableName, false); break; case MERGE_ON_READ: // sync a RO table for MOR - syncHoodieTable(false); - String originalTableName = cfg.tableName; - // TODO : Make realtime table registration optional using a config param - cfg.tableName = cfg.tableName + SUFFIX_REALTIME_TABLE; + syncHoodieTable(roTableTableName.get(), false); // sync a RT table for MOR - syncHoodieTable(true); - cfg.tableName = originalTableName; + syncHoodieTable(snapshotTableName, true); break; default: LOG.error("Unknown table type " + hoodieHiveClient.getTableType()); @@ -89,31 +103,30 @@ public class HiveSyncTool { } } - private void syncHoodieTable(boolean isRealTime) throws ClassNotFoundException { - LOG.info("Trying to sync hoodie table " + cfg.tableName + " with base path " + hoodieHiveClient.getBasePath() + private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat) { + LOG.info("Trying to sync hoodie table " + tableName + " with base path " + hoodieHiveClient.getBasePath() + " of type " + hoodieHiveClient.getTableType()); - // Check if the necessary table exists - boolean tableExists = hoodieHiveClient.doesTableExist(); + boolean tableExists = hoodieHiveClient.doesTableExist(tableName); // Get the parquet schema for this table looking at the latest commit MessageType schema = hoodieHiveClient.getDataSchema(); // Sync schema if needed - syncSchema(tableExists, isRealTime, schema); + syncSchema(tableName, tableExists, useRealtimeInputFormat, schema); - LOG.info("Schema sync complete. Syncing partitions for " + cfg.tableName); + LOG.info("Schema sync complete. Syncing partitions for " + tableName); // Get the last time we successfully synced partitions Option lastCommitTimeSynced = Option.empty(); if (tableExists) { - lastCommitTimeSynced = hoodieHiveClient.getLastCommitTimeSynced(); + lastCommitTimeSynced = hoodieHiveClient.getLastCommitTimeSynced(tableName); } LOG.info("Last commit time synced was found to be " + lastCommitTimeSynced.orElse("null")); List writtenPartitionsSince = hoodieHiveClient.getPartitionsWrittenToSince(lastCommitTimeSynced); LOG.info("Storage partitions scan complete. Found " + writtenPartitionsSince.size()); // Sync the partitions if needed - syncPartitions(writtenPartitionsSince); + syncPartitions(tableName, writtenPartitionsSince); - hoodieHiveClient.updateLastCommitTimeSynced(); - LOG.info("Sync complete for " + cfg.tableName); + hoodieHiveClient.updateLastCommitTimeSynced(tableName); + LOG.info("Sync complete for " + tableName); } /** @@ -123,17 +136,14 @@ public class HiveSyncTool { * @param tableExists - does table exist * @param schema - extracted schema */ - private void syncSchema(boolean tableExists, boolean isRealTime, MessageType schema) throws ClassNotFoundException { + private void syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat, 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) - String inputFormatClassName = - cfg.usePreApacheInputFormat ? com.uber.hoodie.hadoop.HoodieInputFormat.class.getName() - : HoodieParquetInputFormat.class.getName(); - hoodieHiveClient.createTable(schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(), + LOG.info("Hive table " + tableName + " is not found. Creating it"); + if (!useRealTimeInputFormat) { + String inputFormatClassName = cfg.usePreApacheInputFormat ? com.uber.hoodie.hadoop.HoodieInputFormat.class.getName() + : HoodieParquetInputFormat.class.getName(); + hoodieHiveClient.createTable(tableName, schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName()); } else { // Custom serde will not work with ALTER TABLE REPLACE COLUMNS @@ -142,18 +152,18 @@ public class HiveSyncTool { String inputFormatClassName = cfg.usePreApacheInputFormat ? com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat.class.getName() : HoodieParquetRealtimeInputFormat.class.getName(); - hoodieHiveClient.createTable(schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(), + hoodieHiveClient.createTable(tableName, schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName()); } } else { // Check if the table schema has evolved - Map tableSchema = hoodieHiveClient.getTableSchema(); + Map tableSchema = hoodieHiveClient.getTableSchema(tableName); SchemaDifference schemaDiff = SchemaUtil.getSchemaDifference(schema, tableSchema, cfg.partitionFields); if (!schemaDiff.isEmpty()) { - LOG.info("Schema difference found for " + cfg.tableName); - hoodieHiveClient.updateTableDefinition(schema); + LOG.info("Schema difference found for " + tableName); + hoodieHiveClient.updateTableDefinition(tableName, schema); } else { - LOG.info("No Schema difference for " + cfg.tableName); + LOG.info("No Schema difference for " + tableName); } } } @@ -162,19 +172,19 @@ public class HiveSyncTool { * Syncs the list of storage parititions passed in (checks if the partition is in hive, if not adds it or if the * partition path does not match, it updates the partition path). */ - private void syncPartitions(List writtenPartitionsSince) { + private void syncPartitions(String tableName, List writtenPartitionsSince) { try { - List hivePartitions = hoodieHiveClient.scanTablePartitions(); + List hivePartitions = hoodieHiveClient.scanTablePartitions(tableName); List partitionEvents = hoodieHiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince); List newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD); LOG.info("New Partitions " + newPartitions); - hoodieHiveClient.addPartitionsToTable(newPartitions); + hoodieHiveClient.addPartitionsToTable(tableName, newPartitions); List updatePartitions = filterPartitions(partitionEvents, PartitionEventType.UPDATE); LOG.info("Changed Partitions " + updatePartitions); - hoodieHiveClient.updatePartitionsToTable(updatePartitions); + hoodieHiveClient.updatePartitionsToTable(tableName, 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 " + tableName, e); } } diff --git a/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java index 7579f86ae..4578bb2fa 100644 --- a/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java +++ b/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java @@ -18,6 +18,7 @@ package org.apache.hudi.hive; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieLogFile; @@ -135,36 +136,36 @@ public class HoodieHiveClient { /** * Add the (NEW) partitions to the table. */ - void addPartitionsToTable(List partitionsToAdd) { + void addPartitionsToTable(String tableName, List partitionsToAdd) { if (partitionsToAdd.isEmpty()) { - LOG.info("No partitions to add for " + syncConfig.tableName); + LOG.info("No partitions to add for " + tableName); return; } - LOG.info("Adding partitions " + partitionsToAdd.size() + " to table " + syncConfig.tableName); - String sql = constructAddPartitions(partitionsToAdd); + LOG.info("Adding partitions " + partitionsToAdd.size() + " to table " + tableName); + String sql = constructAddPartitions(tableName, partitionsToAdd); updateHiveSQL(sql); } /** * Partition path has changed - update the path for te following partitions. */ - void updatePartitionsToTable(List changedPartitions) { + void updatePartitionsToTable(String tableName, List changedPartitions) { if (changedPartitions.isEmpty()) { - LOG.info("No partitions to change for " + syncConfig.tableName); + LOG.info("No partitions to change for " + tableName); return; } - LOG.info("Changing partitions " + changedPartitions.size() + " on " + syncConfig.tableName); - List sqls = constructChangePartitions(changedPartitions); + LOG.info("Changing partitions " + changedPartitions.size() + " on " + tableName); + List sqls = constructChangePartitions(tableName, changedPartitions); for (String sql : sqls) { updateHiveSQL(sql); } } - private String constructAddPartitions(List partitions) { + private String constructAddPartitions(String tableName, List partitions) { StringBuilder alterSQL = new StringBuilder("ALTER TABLE "); alterSQL.append(HIVE_ESCAPE_CHARACTER).append(syncConfig.databaseName) .append(HIVE_ESCAPE_CHARACTER).append(".").append(HIVE_ESCAPE_CHARACTER) - .append(syncConfig.tableName).append(HIVE_ESCAPE_CHARACTER).append(" ADD IF NOT EXISTS "); + .append(tableName).append(HIVE_ESCAPE_CHARACTER).append(" ADD IF NOT EXISTS "); for (String partition : partitions) { String partitionClause = getPartitionClause(partition); String fullPartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, partition).toString(); @@ -192,12 +193,12 @@ public class HoodieHiveClient { return partBuilder.stream().collect(Collectors.joining(",")); } - private List constructChangePartitions(List partitions) { + private List constructChangePartitions(String tableName, List partitions) { List changePartitions = Lists.newArrayList(); // Hive 2.x doesn't like db.table name for operations, hence we need to change to using the database first String useDatabase = "USE " + HIVE_ESCAPE_CHARACTER + syncConfig.databaseName + HIVE_ESCAPE_CHARACTER; changePartitions.add(useDatabase); - String alterTable = "ALTER TABLE " + HIVE_ESCAPE_CHARACTER + syncConfig.tableName + HIVE_ESCAPE_CHARACTER; + String alterTable = "ALTER TABLE " + HIVE_ESCAPE_CHARACTER + tableName + HIVE_ESCAPE_CHARACTER; for (String partition : partitions) { String partitionClause = getPartitionClause(partition); Path partitionPath = FSUtils.getPartitionPath(syncConfig.basePath, partition); @@ -246,52 +247,52 @@ public class HoodieHiveClient { /** * Scan table partitions. */ - public List scanTablePartitions() throws TException { - return client.listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1); + public List scanTablePartitions(String tableName) throws TException { + return client.listPartitions(syncConfig.databaseName, tableName, (short) -1); } - void updateTableDefinition(MessageType newSchema) { + void updateTableDefinition(String tableName, MessageType newSchema) { try { String newSchemaStr = SchemaUtil.generateSchemaString(newSchema, syncConfig.partitionFields); // Cascade clause should not be present for non-partitioned tables String cascadeClause = syncConfig.partitionFields.size() > 0 ? " cascade" : ""; StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append(HIVE_ESCAPE_CHARACTER) .append(syncConfig.databaseName).append(HIVE_ESCAPE_CHARACTER).append(".") - .append(HIVE_ESCAPE_CHARACTER).append(syncConfig.tableName) + .append(HIVE_ESCAPE_CHARACTER).append(tableName) .append(HIVE_ESCAPE_CHARACTER).append(" REPLACE COLUMNS(") .append(newSchemaStr).append(" )").append(cascadeClause); LOG.info("Updating table definition with " + sqlBuilder); updateHiveSQL(sqlBuilder.toString()); } catch (IOException e) { - throw new HoodieHiveSyncException("Failed to update table for " + syncConfig.tableName, e); + throw new HoodieHiveSyncException("Failed to update table for " + tableName, e); } } - void createTable(MessageType storageSchema, String inputFormatClass, String outputFormatClass, String serdeClass) { + void createTable(String tableName, MessageType storageSchema, String inputFormatClass, String outputFormatClass, String serdeClass) { try { String createSQLQuery = - SchemaUtil.generateCreateDDL(storageSchema, syncConfig, inputFormatClass, outputFormatClass, serdeClass); + SchemaUtil.generateCreateDDL(tableName, storageSchema, syncConfig, inputFormatClass, outputFormatClass, serdeClass); LOG.info("Creating table with " + createSQLQuery); updateHiveSQL(createSQLQuery); } catch (IOException e) { - throw new HoodieHiveSyncException("Failed to create table " + syncConfig.tableName, e); + throw new HoodieHiveSyncException("Failed to create table " + tableName, e); } } /** * Get the table schema. */ - public Map getTableSchema() { + public Map getTableSchema(String tableName) { if (syncConfig.useJdbc) { - if (!doesTableExist()) { + if (!doesTableExist(tableName)) { throw new IllegalArgumentException( - "Failed to get schema for table " + syncConfig.tableName + " does not exist"); + "Failed to get schema for table " + tableName + " does not exist"); } Map schema = Maps.newHashMap(); ResultSet result = null; try { DatabaseMetaData databaseMetaData = connection.getMetaData(); - result = databaseMetaData.getColumns(null, syncConfig.databaseName, syncConfig.tableName, null); + result = databaseMetaData.getColumns(null, syncConfig.databaseName, tableName, null); while (result.next()) { String columnName = result.getString(4); String columnType = result.getString(6); @@ -304,26 +305,26 @@ 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 " + tableName, e); } finally { closeQuietly(result, null); } } else { - return getTableSchemaUsingMetastoreClient(); + return getTableSchemaUsingMetastoreClient(tableName); } } - public Map getTableSchemaUsingMetastoreClient() { + public Map getTableSchemaUsingMetastoreClient(String tableName) { try { // HiveMetastoreClient returns partition keys separate from Columns, hence get both and merge to // get the Schema of the table. final long start = System.currentTimeMillis(); - Table table = this.client.getTable(syncConfig.databaseName, syncConfig.tableName); + Table table = this.client.getTable(syncConfig.databaseName, tableName); Map partitionKeysMap = - table.getPartitionKeys().stream().collect(Collectors.toMap(f -> f.getName(), f -> f.getType().toUpperCase())); + table.getPartitionKeys().stream().collect(Collectors.toMap(FieldSchema::getName, f -> f.getType().toUpperCase())); Map columnsMap = - table.getSd().getCols().stream().collect(Collectors.toMap(f -> f.getName(), f -> f.getType().toUpperCase())); + table.getSd().getCols().stream().collect(Collectors.toMap(FieldSchema::getName, f -> f.getType().toUpperCase())); Map schema = new HashMap<>(); schema.putAll(columnsMap); @@ -332,7 +333,7 @@ public class HoodieHiveClient { LOG.info(String.format("Time taken to getTableSchema: %s ms", (end - start))); return schema; } catch (Exception e) { - throw new HoodieHiveSyncException("Failed to get table schema for : " + syncConfig.tableName, e); + throw new HoodieHiveSyncException("Failed to get table schema for : " + tableName, e); } } @@ -357,7 +358,7 @@ public class HoodieHiveClient { .orElseThrow(() -> new IllegalArgumentException("Could not find any data file written for commit " + lastCommit + ", could not get schema for table " + metaClient.getBasePath() + ", Metadata :" + commitMetadata)); - return readSchemaFromDataFile(new Path(filePath)); + return readSchemaFromBaseFile(new Path(filePath)); case MERGE_ON_READ: // If this is MOR, depending on whether the latest commit is a delta commit or // compaction commit @@ -387,18 +388,17 @@ public class HoodieHiveClient { .map(f -> Pair.of(f, HoodieFileFormat.HOODIE_LOG)).orElseGet(() -> { // No Log files in Delta-Commit. Check if there are any parquet files return commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream() - .filter(s -> s.contains((metaClient.getTableConfig().getROFileFormat().getFileExtension()))) - .findAny().map(f -> Pair.of(f, HoodieFileFormat.PARQUET)).orElseThrow(() -> { - return new IllegalArgumentException("Could not find any data file written for commit " - + lastDeltaInstant + ", could not get schema for table " + metaClient.getBasePath() - + ", CommitMetadata :" + commitMetadata); - }); + .filter(s -> s.contains((metaClient.getTableConfig().getBaseFileFormat().getFileExtension()))) + .findAny().map(f -> Pair.of(f, HoodieFileFormat.PARQUET)).orElseThrow(() -> + new IllegalArgumentException("Could not find any data file written for commit " + + lastDeltaInstant + ", could not get schema for table " + metaClient.getBasePath() + + ", CommitMetadata :" + commitMetadata)); }); switch (filePathWithFormat.getRight()) { case HOODIE_LOG: return readSchemaFromLogFile(lastCompactionCommit, new Path(filePathWithFormat.getLeft())); case PARQUET: - return readSchemaFromDataFile(new Path(filePathWithFormat.getLeft())); + return readSchemaFromBaseFile(new Path(filePathWithFormat.getLeft())); default: throw new IllegalArgumentException("Unknown file format :" + filePathWithFormat.getRight() + " for file " + filePathWithFormat.getLeft()); @@ -411,7 +411,7 @@ public class HoodieHiveClient { throw new InvalidTableException(syncConfig.basePath); } } catch (IOException e) { - throw new HoodieHiveSyncException("Failed to get table schema for " + syncConfig.tableName, e); + throw new HoodieHiveSyncException("Failed to read data schema", e); } } @@ -429,7 +429,7 @@ public class HoodieHiveClient { String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny() .orElseThrow(() -> new IllegalArgumentException("Could not find any data file written for compaction " + lastCompactionCommit + ", could not get schema for table " + metaClient.getBasePath())); - return readSchemaFromDataFile(new Path(filePath)); + return readSchemaFromBaseFile(new Path(filePath)); } /** @@ -450,7 +450,7 @@ public class HoodieHiveClient { /** * Read the parquet schema from a parquet File. */ - private MessageType readSchemaFromDataFile(Path parquetFilePath) throws IOException { + private MessageType readSchemaFromBaseFile(Path parquetFilePath) throws IOException { LOG.info("Reading schema from " + parquetFilePath); if (!fs.exists(parquetFilePath)) { throw new IllegalArgumentException( @@ -464,11 +464,11 @@ public class HoodieHiveClient { /** * @return true if the configured table exists */ - public boolean doesTableExist() { + public boolean doesTableExist(String tableName) { try { - return client.tableExists(syncConfig.databaseName, syncConfig.tableName); + return client.tableExists(syncConfig.databaseName, 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 " + tableName, e); } } @@ -603,10 +603,10 @@ public class HoodieHiveClient { return fs; } - public Option getLastCommitTimeSynced() { + public Option getLastCommitTimeSynced(String tableName) { // Get the last commit time from the TBLproperties try { - Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName); + Table database = client.getTable(syncConfig.databaseName, tableName); return Option.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); @@ -654,17 +654,16 @@ public class HoodieHiveClient { return client.getAllTables(db); } - void updateLastCommitTimeSynced() { + void updateLastCommitTimeSynced(String tableName) { // Set the last commit time from the TBLproperties String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp(); try { - Table table = client.getTable(syncConfig.databaseName, syncConfig.tableName); + Table table = client.getTable(syncConfig.databaseName, tableName); table.putToParameters(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced); - client.alter_table(syncConfig.databaseName, syncConfig.tableName, table); + client.alter_table(syncConfig.databaseName, tableName, table); } catch (Exception e) { throw new HoodieHiveSyncException("Failed to get update last commit time synced to " + lastCommitSynced, e); } - } /** diff --git a/hudi-hive/src/main/java/org/apache/hudi/hive/util/SchemaUtil.java b/hudi-hive/src/main/java/org/apache/hudi/hive/util/SchemaUtil.java index 16d9aae64..d85778a3a 100644 --- a/hudi-hive/src/main/java/org/apache/hudi/hive/util/SchemaUtil.java +++ b/hudi-hive/src/main/java/org/apache/hudi/hive/util/SchemaUtil.java @@ -391,7 +391,7 @@ public class SchemaUtil { return columns.toString(); } - public static String generateCreateDDL(MessageType storageSchema, HiveSyncConfig config, String inputFormatClass, + public static String generateCreateDDL(String tableName, MessageType storageSchema, HiveSyncConfig config, String inputFormatClass, String outputFormatClass, String serdeClass) throws IOException { Map hiveSchema = convertParquetSchemaToHiveSchema(storageSchema); String columns = generateSchemaString(storageSchema, config.partitionFields); @@ -406,7 +406,7 @@ public class SchemaUtil { String partitionsStr = partitionFields.stream().collect(Collectors.joining(",")); StringBuilder sb = new StringBuilder("CREATE EXTERNAL TABLE IF NOT EXISTS "); sb = sb.append(HIVE_ESCAPE_CHARACTER).append(config.databaseName).append(HIVE_ESCAPE_CHARACTER) - .append(".").append(HIVE_ESCAPE_CHARACTER).append(config.tableName).append(HIVE_ESCAPE_CHARACTER); + .append(".").append(HIVE_ESCAPE_CHARACTER).append(tableName).append(HIVE_ESCAPE_CHARACTER); sb = sb.append("( ").append(columns).append(")"); if (!config.partitionFields.isEmpty()) { sb = sb.append(" PARTITIONED BY (").append(partitionsStr).append(")"); diff --git a/hudi-hive/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-hive/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index 6639ea2fe..50bb0e5d3 100644 --- a/hudi-hive/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-hive/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -158,18 +158,19 @@ public class TestHiveSyncTool { HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially", - hiveClient.doesTableExist()); + hiveClient.doesTableExist(TestUtil.hiveSyncConfig.tableName)); // Lets do the sync HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + " should exist after sync completes", - hiveClient.doesTableExist()); - assertEquals("Hive Schema should match the table schema + partition field", hiveClient.getTableSchema().size(), + hiveClient.doesTableExist(TestUtil.hiveSyncConfig.tableName)); + assertEquals("Hive Schema should match the table schema + partition field", + hiveClient.getTableSchema(TestUtil.hiveSyncConfig.tableName).size(), hiveClient.getDataSchema().getColumns().size() + 1); assertEquals("Table partitions should match the number of partitions we wrote", 5, - hiveClient.scanTablePartitions().size()); + hiveClient.scanTablePartitions(TestUtil.hiveSyncConfig.tableName).size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", commitTime, - hiveClient.getLastCommitTimeSynced().get()); + hiveClient.getLastCommitTimeSynced(TestUtil.hiveSyncConfig.tableName).get()); } @Test @@ -183,9 +184,9 @@ public class TestHiveSyncTool { HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); assertEquals("Table partitions should match the number of partitions we wrote", 5, - hiveClient.scanTablePartitions().size()); + hiveClient.scanTablePartitions(TestUtil.hiveSyncConfig.tableName).size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", commitTime1, - hiveClient.getLastCommitTimeSynced().get()); + hiveClient.getLastCommitTimeSynced(TestUtil.hiveSyncConfig.tableName).get()); // Now lets create more parititions and these are the only ones which needs to be synced DateTime dateTime = DateTime.now().plusDays(6); @@ -196,7 +197,7 @@ public class TestHiveSyncTool { hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.of(commitTime1)); assertEquals("We should have one partition written after 100 commit", 1, writtenPartitionsSince.size()); - List hivePartitions = hiveClient.scanTablePartitions(); + List hivePartitions = hiveClient.scanTablePartitions(TestUtil.hiveSyncConfig.tableName); 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, @@ -205,9 +206,10 @@ public class TestHiveSyncTool { 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 one partition we wrote should be added to hive", 6, + hiveClient.scanTablePartitions(TestUtil.hiveSyncConfig.tableName).size()); assertEquals("The last commit that was sycned should be 101", commitTime2, - hiveClient.getLastCommitTimeSynced().get()); + hiveClient.getLastCommitTimeSynced(TestUtil.hiveSyncConfig.tableName).get()); } @Test @@ -221,7 +223,7 @@ public class TestHiveSyncTool { HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); - int fields = hiveClient.getTableSchema().size(); + int fields = hiveClient.getTableSchema(TestUtil.hiveSyncConfig.tableName).size(); // Now lets create more parititions and these are the only ones which needs to be synced DateTime dateTime = DateTime.now().plusDays(6); @@ -233,16 +235,17 @@ public class TestHiveSyncTool { tool.syncHoodieTable(); assertEquals("Hive Schema has evolved and should not be 3 more field", fields + 3, - hiveClient.getTableSchema().size()); + hiveClient.getTableSchema(TestUtil.hiveSyncConfig.tableName).size()); assertEquals("Hive Schema has evolved - Field favorite_number has evolved from int to long", "BIGINT", - hiveClient.getTableSchema().get("favorite_number")); + hiveClient.getTableSchema(TestUtil.hiveSyncConfig.tableName).get("favorite_number")); assertTrue("Hive Schema has evolved - Field favorite_movie was added", - hiveClient.getTableSchema().containsKey("favorite_movie")); + hiveClient.getTableSchema(TestUtil.hiveSyncConfig.tableName).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 one partition we wrote should be added to hive", 6, + hiveClient.scanTablePartitions(TestUtil.hiveSyncConfig.tableName).size()); assertEquals("The last commit that was sycned should be 101", commitTime2, - hiveClient.getLastCommitTimeSynced().get()); + hiveClient.getLastCommitTimeSynced(TestUtil.hiveSyncConfig.tableName).get()); } @Test @@ -251,24 +254,24 @@ public class TestHiveSyncTool { String commitTime = "100"; String deltaCommitTime = "101"; TestUtil.createMORTable(commitTime, deltaCommitTime, 5); - HoodieHiveClient hiveClient = - new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); - assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially", - hiveClient.doesTableExist()); + + String roTableName = TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE; + HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); + assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially", hiveClient.doesTableExist(roTableName)); // Lets do the sync HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); - assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + " should exist after sync completes", - hiveClient.doesTableExist()); - assertEquals("Hive Schema should match the table schema + partition field", hiveClient.getTableSchema().size(), + assertTrue("Table " + roTableName + " should exist after sync completes", + hiveClient.doesTableExist(roTableName)); + assertEquals("Hive Schema should match the table schema + partition field", hiveClient.getTableSchema(roTableName).size(), SchemaTestUtil.getSimpleSchema().getFields().size() + 1); assertEquals("Table partitions should match the number of partitions we wrote", 5, - hiveClient.scanTablePartitions().size()); + hiveClient.scanTablePartitions(roTableName).size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", deltaCommitTime, - hiveClient.getLastCommitTimeSynced().get()); + hiveClient.getLastCommitTimeSynced(roTableName).get()); - // Now lets create more parititions and these are the only ones which needs to be synced + // Now lets create more partitions and these are the only ones which needs to be synced DateTime dateTime = DateTime.now().plusDays(6); String commitTime2 = "102"; String deltaCommitTime2 = "103"; @@ -281,11 +284,11 @@ public class TestHiveSyncTool { hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); assertEquals("Hive Schema should match the evolved table schema + partition field", - hiveClient.getTableSchema().size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + 1); + hiveClient.getTableSchema(roTableName).size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + 1); // Sync should add the one partition - assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClient.scanTablePartitions().size()); + assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClient.scanTablePartitions(roTableName).size()); assertEquals("The last commit that was synced should be 103", deltaCommitTime2, - hiveClient.getLastCommitTimeSynced().get()); + hiveClient.getLastCommitTimeSynced(roTableName).get()); } @Test @@ -293,28 +296,27 @@ public class TestHiveSyncTool { TestUtil.hiveSyncConfig.useJdbc = this.useJdbc; String commitTime = "100"; String deltaCommitTime = "101"; - String roTablename = TestUtil.hiveSyncConfig.tableName; - TestUtil.hiveSyncConfig.tableName = TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE; + String snapshotTableName = TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; TestUtil.createMORTable(commitTime, deltaCommitTime, 5); HoodieHiveClient hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); - assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE - + " should not exist initially", hiveClientRT.doesTableExist()); + assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + + " should not exist initially", hiveClientRT.doesTableExist(snapshotTableName)); // Lets do the sync HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); - assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE - + " should exist after sync completes", hiveClientRT.doesTableExist()); + assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + + " should exist after sync completes", hiveClientRT.doesTableExist(snapshotTableName)); - assertEquals("Hive Schema should match the table schema + partition field", hiveClientRT.getTableSchema().size(), + assertEquals("Hive Schema should match the table schema + partition field", hiveClientRT.getTableSchema(snapshotTableName).size(), SchemaTestUtil.getSimpleSchema().getFields().size() + 1); assertEquals("Table partitions should match the number of partitions we wrote", 5, - hiveClientRT.scanTablePartitions().size()); + hiveClientRT.scanTablePartitions(snapshotTableName).size()); assertEquals("The last commit that was synced should be updated in the TBLPROPERTIES", deltaCommitTime, - hiveClientRT.getLastCommitTimeSynced().get()); + hiveClientRT.getLastCommitTimeSynced(snapshotTableName).get()); // Now lets create more parititions and these are the only ones which needs to be synced DateTime dateTime = DateTime.now().plusDays(6); @@ -329,12 +331,11 @@ public class TestHiveSyncTool { hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); assertEquals("Hive Schema should match the evolved table schema + partition field", - hiveClientRT.getTableSchema().size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + 1); + hiveClientRT.getTableSchema(snapshotTableName).size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + 1); // Sync should add the one partition - assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClientRT.scanTablePartitions().size()); + assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClientRT.scanTablePartitions(snapshotTableName).size()); assertEquals("The last commit that was sycned should be 103", deltaCommitTime2, - hiveClientRT.getLastCommitTimeSynced().get()); - TestUtil.hiveSyncConfig.tableName = roTablename; + hiveClientRT.getLastCommitTimeSynced(snapshotTableName).get()); } @Test @@ -350,16 +351,19 @@ public class TestHiveSyncTool { TestUtil.getCreatedTablesSet().add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); - assertFalse("Table " + hiveSyncConfig.tableName + " should not exist initially", hiveClient.doesTableExist()); + assertFalse("Table " + hiveSyncConfig.tableName + " should not exist initially", + hiveClient.doesTableExist(hiveSyncConfig.tableName)); // Lets do the sync HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); tool.syncHoodieTable(); - assertTrue("Table " + hiveSyncConfig.tableName + " should exist after sync completes", hiveClient.doesTableExist()); - assertEquals("Hive Schema should match the table schema + partition fields", hiveClient.getTableSchema().size(), + assertTrue("Table " + hiveSyncConfig.tableName + " should exist after sync completes", + hiveClient.doesTableExist(hiveSyncConfig.tableName)); + assertEquals("Hive Schema should match the table schema + partition fields", + hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), hiveClient.getDataSchema().getColumns().size() + 3); assertEquals("Table partitions should match the number of partitions we wrote", 5, - hiveClient.scanTablePartitions().size()); + hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", commitTime, - hiveClient.getLastCommitTimeSynced().get()); + hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get()); } } diff --git a/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java b/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java index 425854a08..0cec024c9 100644 --- a/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java +++ b/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java @@ -26,7 +26,7 @@ import org.apache.hudi.common.minicluster.HdfsTestService; import org.apache.hudi.common.minicluster.ZookeeperTestService; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieDeltaWriteStat; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTableType; @@ -179,7 +179,7 @@ public class TestUtil { HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true, dateTime, commitTime); createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); createdTablesSet - .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE); + .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() .forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l))); @@ -202,9 +202,8 @@ public class TestUtil { throws IOException, URISyntaxException, InterruptedException { HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, isParquetSchemaSimple, startFrom, commitTime); - createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); - createdTablesSet - .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE); + createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE); + createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() .forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l))); @@ -220,7 +219,7 @@ public class TestUtil { String partitionPath = wEntry.getKey(); for (HoodieWriteStat wStat : wEntry.getValue()) { Path path = new Path(wStat.getPath()); - HoodieDataFile dataFile = new HoodieDataFile(fileSystem.getFileStatus(path)); + HoodieBaseFile dataFile = new HoodieBaseFile(fileSystem.getFileStatus(path)); HoodieLogFile logFile = generateLogData(path, isLogSchemaSimple); HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat(); writeStat.setFileId(dataFile.getFileId()); @@ -291,7 +290,7 @@ public class TestUtil { private static HoodieLogFile generateLogData(Path parquetFilePath, boolean isLogSchemaSimple) throws IOException, InterruptedException, URISyntaxException { Schema schema = (isLogSchemaSimple ? SchemaTestUtil.getSimpleSchema() : SchemaTestUtil.getEvolvedSchema()); - HoodieDataFile dataFile = new HoodieDataFile(fileSystem.getFileStatus(parquetFilePath)); + HoodieBaseFile dataFile = new HoodieBaseFile(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()) diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java index 99a8d010e..7d1141455 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java @@ -118,14 +118,14 @@ public class ITTestHoodieDemo extends ITTestBase { private void ingestFirstBatchAndHiveSync() throws Exception { List cmds = new ImmutableList.Builder() .add("spark-submit --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer " + HUDI_UTILITIES_BUNDLE - + " --storage-type COPY_ON_WRITE " + + " --table-type COPY_ON_WRITE " + " --source-class org.apache.hudi.utilities.sources.JsonDFSSource --source-ordering-field ts " + " --target-base-path " + COW_BASE_PATH + " --target-table " + COW_TABLE_NAME + " --props /var/demo/config/dfs-source.properties " + " --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider " + String.format(HIVE_SYNC_CMD_FMT, "dt", COW_TABLE_NAME)) .add("spark-submit --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer " + HUDI_UTILITIES_BUNDLE - + " --storage-type MERGE_ON_READ " + + " --table-type MERGE_ON_READ " + " --source-class org.apache.hudi.utilities.sources.JsonDFSSource --source-ordering-field ts " + " --target-base-path " + MOR_BASE_PATH + " --target-table " + MOR_TABLE_NAME + " --props /var/demo/config/dfs-source.properties " @@ -139,7 +139,7 @@ public class ITTestHoodieDemo extends ITTestBase { private void testHiveAfterFirstBatch() throws Exception { Pair stdOutErrPair = executeHiveCommandFile(HIVE_TBLCHECK_COMMANDS); assertStdOutContains(stdOutErrPair, "| stock_ticks_cow |"); - assertStdOutContains(stdOutErrPair, "| stock_ticks_mor |"); + assertStdOutContains(stdOutErrPair, "| stock_ticks_mor_ro |"); assertStdOutContains(stdOutErrPair, "| stock_ticks_mor_rt |"); assertStdOutContains(stdOutErrPair, @@ -159,7 +159,8 @@ public class ITTestHoodieDemo extends ITTestBase { private void testSparkSQLAfterFirstBatch() throws Exception { Pair stdOutErrPair = executeSparkSQLCommand(SPARKSQL_BATCH1_COMMANDS, true); assertStdOutContains(stdOutErrPair, "|default |stock_ticks_cow |false |\n" - + "|default |stock_ticks_mor |false |\n|default |stock_ticks_mor_rt |false |"); + + "|default |stock_ticks_mor_ro |false |\n" + + "|default |stock_ticks_mor_rt |false |"); assertStdOutContains(stdOutErrPair, "+------+-------------------+\n|GOOG |2018-08-31 10:29:00|\n+------+-------------------+", 3); assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 09:59:00|6330 |1230.5 |1230.02 |", 3); @@ -170,14 +171,14 @@ public class ITTestHoodieDemo extends ITTestBase { List cmds = new ImmutableList.Builder() .add("hdfs dfs -copyFromLocal -f " + INPUT_BATCH_PATH2 + " " + HDFS_BATCH_PATH2) .add("spark-submit --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer " + HUDI_UTILITIES_BUNDLE - + " --storage-type COPY_ON_WRITE " + + " --table-type COPY_ON_WRITE " + " --source-class org.apache.hudi.utilities.sources.JsonDFSSource --source-ordering-field ts " + " --target-base-path " + COW_BASE_PATH + " --target-table " + COW_TABLE_NAME + " --props /var/demo/config/dfs-source.properties " + " --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider " + String.format(HIVE_SYNC_CMD_FMT, "dt", COW_TABLE_NAME)) .add("spark-submit --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer " + HUDI_UTILITIES_BUNDLE - + " --storage-type MERGE_ON_READ " + + " --table-type MERGE_ON_READ " + " --source-class org.apache.hudi.utilities.sources.JsonDFSSource --source-ordering-field ts " + " --target-base-path " + MOR_BASE_PATH + " --target-table " + MOR_TABLE_NAME + " --props /var/demo/config/dfs-source.properties " @@ -291,8 +292,10 @@ public class ITTestHoodieDemo extends ITTestBase { Pair stdOutErrPair = executeSparkSQLCommand(SPARKSQL_INCREMENTAL_COMMANDS, true); assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 10:59:00|9021 |1227.1993|1227.215|"); assertStdOutContains(stdOutErrPair, "|default |stock_ticks_cow |false |\n" - + "|default |stock_ticks_derived_mor |false |\n|default |stock_ticks_derived_mor_rt|false |\n" - + "|default |stock_ticks_mor |false |\n|default |stock_ticks_mor_rt |false |\n" + + "|default |stock_ticks_derived_mor_ro|false |\n" + + "|default |stock_ticks_derived_mor_rt|false |\n" + + "|default |stock_ticks_mor_ro |false |\n" + + "|default |stock_ticks_mor_rt |false |\n" + "| |stock_ticks_cow_incr |true |"); assertStdOutContains(stdOutErrPair, "|count(1)|\n+--------+\n|99 |", 2); } diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieSanity.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieSanity.java index ac54df949..4b6d89f9e 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieSanity.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieSanity.java @@ -18,6 +18,7 @@ package org.apache.hudi.integ; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.model.HoodieTableType; @@ -144,38 +145,46 @@ public class ITTestHoodieSanity extends ITTestBase { } executeCommandStringInDocker(ADHOC_1_CONTAINER, cmd, true); + String snapshotTableName = tableType.equals(HoodieTableType.MERGE_ON_READ.name()) ? + hiveTableName + "_rt" : hiveTableName; + Option roTableName = tableType.equals(HoodieTableType.MERGE_ON_READ.name()) ? + Option.of(hiveTableName +"_ro") : Option.empty(); + // Ensure table does exist - stdOutErr = executeHiveCommand("show tables like '" + hiveTableName + "'"); - Assert.assertEquals("Table exists", hiveTableName, stdOutErr.getLeft()); + stdOutErr = executeHiveCommand("show tables like '" + snapshotTableName + "'"); + Assert.assertEquals("Table exists", snapshotTableName, stdOutErr.getLeft()); // Ensure row count is 80 (without duplicates) (100 - 20 deleted) - stdOutErr = executeHiveCommand("select count(1) from " + hiveTableName); - Assert.assertEquals("Expecting 100 rows to be present in the new table", 80, + stdOutErr = executeHiveCommand("select count(1) from " + snapshotTableName); + Assert.assertEquals("Expecting 80 rows to be present in the snapshot table", 80, Integer.parseInt(stdOutErr.getLeft().trim())); - // If is MOR table, ensure realtime table row count is 100 - 20 = 80 (without duplicates) - if (tableType.equals(HoodieTableType.MERGE_ON_READ.name())) { - stdOutErr = executeHiveCommand("select count(1) from " + hiveTableName + "_rt"); - Assert.assertEquals("Expecting 100 rows to be present in the realtime table,", 80, + if (roTableName.isPresent()) { + stdOutErr = executeHiveCommand("select count(1) from " + roTableName.get()); + Assert.assertEquals("Expecting 80 rows to be present in the snapshot table", 80, Integer.parseInt(stdOutErr.getLeft().trim())); } - // Make the HDFS dataset non-hoodie and run the same query - // Checks for interoperability with non-hoodie tables - + // Make the HDFS dataset non-hoodie and run the same query; Checks for interoperability with non-hoodie tables // Delete Hoodie directory to make it non-hoodie dataset executeCommandStringInDocker(ADHOC_1_CONTAINER, "hdfs dfs -rm -r " + hdfsPath + "/.hoodie", true); // Run the count query again. Without Hoodie, all versions are included. So we get a wrong count - stdOutErr = executeHiveCommand("select count(1) from " + hiveTableName); + if (tableType.equals(HoodieTableType.MERGE_ON_READ.name())) { + stdOutErr = executeHiveCommand("select count(1) from " + roTableName.get()); + } else { + stdOutErr = executeHiveCommand("select count(1) from " + snapshotTableName); + } Assert.assertEquals("Expecting 280 rows to be present in the new table", 280, Integer.parseInt(stdOutErr.getLeft().trim())); } private void dropHiveTables(String hiveTableName, String tableType) throws Exception { - executeHiveCommand("drop table if exists " + hiveTableName); if (tableType.equals(HoodieTableType.MERGE_ON_READ.name())) { executeHiveCommand("drop table if exists " + hiveTableName + "_rt"); + executeHiveCommand("drop table if exists " + hiveTableName + "_ro"); + } else { + executeHiveCommand("drop table if exists " + hiveTableName); } } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java index 83bad6f65..af8bb1697 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -138,8 +138,8 @@ public class DataSourceUtils { String tblName, Map parameters) { // inline compaction is on by default for MOR - boolean inlineCompact = parameters.get(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY()) - .equals(DataSourceWriteOptions.MOR_STORAGE_TYPE_OPT_VAL()); + boolean inlineCompact = parameters.get(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY()) + .equals(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL()); // insert/bulk-insert combining to be true, if filtering for duplicates boolean combineInserts = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY())); diff --git a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 564e2a47a..302ca755a 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -21,6 +21,7 @@ import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor import org.apache.hudi.keygen.SimpleKeyGenerator +import org.apache.log4j.LogManager /** * List of options that can be passed to the Hoodie datasource, @@ -31,20 +32,49 @@ import org.apache.hudi.keygen.SimpleKeyGenerator * Options supported for reading hoodie tables. */ object DataSourceReadOptions { + + private val log = LogManager.getLogger(classOf[DefaultSource]) + /** * Whether data needs to be read, in - * incremental mode (new data since an instantTime) - * (or) Read Optimized mode (obtain latest view, based on columnar data) - * (or) Real time mode (obtain latest view, based on row & columnar data) * - * Default: READ_OPTIMIZED + * 1) Snapshot mode (obtain latest view, based on row & columnar data) + * 2) incremental mode (new data since an instantTime) + * 3) Read Optimized mode (obtain latest view, based on columnar data) + * + * Default: snapshot */ + val QUERY_TYPE_OPT_KEY = "hoodie.datasource.query.type" + val QUERY_TYPE_SNAPSHOT_OPT_VAL = "snapshot" + val QUERY_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized" + val QUERY_TYPE_INCREMENTAL_OPT_VAL = "incremental" + val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_SNAPSHOT_OPT_VAL + + @Deprecated val VIEW_TYPE_OPT_KEY = "hoodie.datasource.view.type" + @Deprecated val VIEW_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized" + @Deprecated val VIEW_TYPE_INCREMENTAL_OPT_VAL = "incremental" + @Deprecated val VIEW_TYPE_REALTIME_OPT_VAL = "realtime" + @Deprecated val DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL - val DEFAULTPUSH_DOWN_FILTERS_OPT_VAL = "" + + /** + * This eases migration from old configs to new configs. + */ + def translateViewTypesToQueryTypes(optParams: Map[String, String]) : Map[String, String] = { + val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL, + VIEW_TYPE_INCREMENTAL_OPT_VAL -> QUERY_TYPE_INCREMENTAL_OPT_VAL, + VIEW_TYPE_REALTIME_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL) + if (optParams.contains(VIEW_TYPE_OPT_KEY) && !optParams.contains(QUERY_TYPE_OPT_KEY)) { + log.warn(VIEW_TYPE_OPT_KEY + " is deprecated and will be removed in a later release. Please use " + QUERY_TYPE_OPT_KEY) + optParams ++ Map(QUERY_TYPE_OPT_KEY -> translation(optParams(VIEW_TYPE_OPT_KEY))) + } else { + optParams + } + } /** * Instant time to start incrementally pulling data from. The instanttime here need not @@ -72,14 +102,18 @@ object DataSourceReadOptions { * This option allows setting filters directly on Hoodie Source */ val PUSH_DOWN_INCR_FILTERS_OPT_KEY = "hoodie.datasource.read.incr.filters" + val DEFAULTPUSH_DOWN_FILTERS_OPT_VAL = "" } /** * Options supported for writing hoodie tables. */ object DataSourceWriteOptions { + + private val log = LogManager.getLogger(classOf[DefaultSource]) + /** - * The client operation, that this write should do + * The write operation, that this write should do * * Default: upsert() */ @@ -91,16 +125,35 @@ object DataSourceWriteOptions { val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL /** - * The storage type for the underlying data, for this write. + * The table type for the underlying data, for this write. * Note that this can't change across writes. * * Default: COPY_ON_WRITE */ + val TABLE_TYPE_OPT_KEY = "hoodie.datasource.write.table.type" + val COW_TABLE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name + val MOR_TABLE_TYPE_OPT_VAL = HoodieTableType.MERGE_ON_READ.name + val DEFAULT_TABLE_TYPE_OPT_VAL = COW_TABLE_TYPE_OPT_VAL + + @Deprecated val STORAGE_TYPE_OPT_KEY = "hoodie.datasource.write.storage.type" + @Deprecated val COW_STORAGE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name + @Deprecated val MOR_STORAGE_TYPE_OPT_VAL = HoodieTableType.MERGE_ON_READ.name + @Deprecated val DEFAULT_STORAGE_TYPE_OPT_VAL = COW_STORAGE_TYPE_OPT_VAL + def translateStorageTypeToTableType(optParams: Map[String, String]) : Map[String, String] = { + if (optParams.contains(STORAGE_TYPE_OPT_KEY) && !optParams.contains(TABLE_TYPE_OPT_KEY)) { + log.warn(STORAGE_TYPE_OPT_KEY + " is deprecated and will be removed in a later release; Please use " + TABLE_TYPE_OPT_KEY) + optParams ++ Map(TABLE_TYPE_OPT_KEY -> optParams(STORAGE_TYPE_OPT_KEY)) + } else { + optParams + } + } + + /** * Hive table name, to register the table into. * diff --git a/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala index 10321f32d..23a757db9 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -50,20 +50,14 @@ class DefaultSource extends RelationProvider optParams: Map[String, String], schema: StructType): BaseRelation = { // Add default options for unspecified read options keys. - val parameters = Map(VIEW_TYPE_OPT_KEY -> DEFAULT_VIEW_TYPE_OPT_VAL) ++ optParams + val parameters = Map(QUERY_TYPE_OPT_KEY -> DEFAULT_QUERY_TYPE_OPT_VAL) ++ translateViewTypesToQueryTypes(optParams) val path = parameters.get("path") if (path.isEmpty) { throw new HoodieException("'path' must be specified.") } - if (parameters(VIEW_TYPE_OPT_KEY).equals(VIEW_TYPE_REALTIME_OPT_VAL)) { - throw new HoodieException("Realtime view not supported yet via data source. Please use HiveContext route.") - } - - if (parameters(VIEW_TYPE_OPT_KEY).equals(VIEW_TYPE_INCREMENTAL_OPT_VAL)) { - new IncrementalRelation(sqlContext, path.get, optParams, schema) - } else { + if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_SNAPSHOT_OPT_VAL)) { // this is just effectively RO view only, where `path` can contain a mix of // non-hoodie/hoodie path files. set the path filter up sqlContext.sparkContext.hadoopConfiguration.setClass( @@ -72,6 +66,8 @@ class DefaultSource extends RelationProvider classOf[org.apache.hadoop.fs.PathFilter]); log.info("Constructing hoodie (as parquet) data source with options :" + parameters) + log.warn("Snapshot view not supported yet via data source, for MERGE_ON_READ tables. " + + "Please query the Hive table registered using Spark SQL.") // simply return as a regular parquet relation DataSource.apply( sparkSession = sqlContext.sparkSession, @@ -79,6 +75,10 @@ class DefaultSource extends RelationProvider className = "parquet", options = parameters) .resolveRelation() + } else if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_INCREMENTAL_OPT_VAL)) { + new IncrementalRelation(sqlContext, path.get, optParams, schema) + } else { + throw new HoodieException("Invalid query type :" + parameters(QUERY_TYPE_OPT_KEY)) } } diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 5e6100280..d4d48f960 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -58,7 +58,7 @@ private[hudi] object HoodieSparkSqlWriter { case Some(ser) if ser.equals("org.apache.spark.serializer.KryoSerializer") => case _ => throw new HoodieException("hoodie only support org.apache.spark.serializer.KryoSerializer as spark.serializer") } - val storageType = parameters(STORAGE_TYPE_OPT_KEY) + val tableType = parameters(TABLE_TYPE_OPT_KEY) val operation = // It does not make sense to allow upsert() operation if INSERT_DROP_DUPS_OPT_KEY is true // Auto-correct the operation to "insert" if OPERATION_OPT_KEY is set to "upsert" wrongly @@ -119,7 +119,7 @@ private[hudi] object HoodieSparkSqlWriter { // Create the table if not present if (!exists) { - HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, storageType, + HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, tableType, tblName.get, "archived", parameters(PAYLOAD_CLASS_OPT_KEY)) } @@ -192,7 +192,7 @@ private[hudi] object HoodieSparkSqlWriter { */ def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = { Map(OPERATION_OPT_KEY -> DEFAULT_OPERATION_OPT_VAL, - STORAGE_TYPE_OPT_KEY -> DEFAULT_STORAGE_TYPE_OPT_VAL, + TABLE_TYPE_OPT_KEY -> DEFAULT_TABLE_TYPE_OPT_VAL, PRECOMBINE_FIELD_OPT_KEY -> DEFAULT_PRECOMBINE_FIELD_OPT_VAL, PAYLOAD_CLASS_OPT_KEY -> DEFAULT_PAYLOAD_OPT_VAL, RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL, @@ -212,7 +212,7 @@ private[hudi] object HoodieSparkSqlWriter { HIVE_PARTITION_FIELDS_OPT_KEY -> DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL, HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL, HIVE_STYLE_PARTITIONING_OPT_KEY -> DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL - ) ++ parameters + ) ++ translateStorageTypeToTableType(parameters) } def toProperties(params: Map[String, String]): TypedProperties = { diff --git a/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark/src/test/java/HoodieJavaApp.java index f44cd0a9d..978d35934 100644 --- a/hudi-spark/src/test/java/HoodieJavaApp.java +++ b/hudi-spark/src/test/java/HoodieJavaApp.java @@ -132,7 +132,7 @@ public class HoodieJavaApp { // full list in HoodieWriteConfig & its package .option("hoodie.upsert.shuffle.parallelism", "2") // Hoodie Table Type - .option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType) // insert .option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL()) // This is the record key @@ -165,7 +165,7 @@ public class HoodieJavaApp { Dataset inputDF2 = spark.read().json(jssc.parallelize(records2, 2)); writer = inputDF2.write().format("org.apache.hudi").option("hoodie.insert.shuffle.parallelism", "2") .option("hoodie.upsert.shuffle.parallelism", "2") - .option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") @@ -188,7 +188,7 @@ public class HoodieJavaApp { Dataset inputDF3 = spark.read().json(jssc.parallelize(deletes, 2)); writer = inputDF3.write().format("org.apache.hudi").option("hoodie.insert.shuffle.parallelism", "2") .option("hoodie.upsert.shuffle.parallelism", "2") - .option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type .option(DataSourceWriteOptions.OPERATION_OPT_KEY(), "delete") .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") @@ -206,11 +206,11 @@ public class HoodieJavaApp { /** * Read & do some queries */ - Dataset hoodieROViewDF = spark.read().format("org.apache.hudi") + Dataset snapshotQueryDF = spark.read().format("org.apache.hudi") // pass any path glob, can include hoodie & non-hoodie // datasets .load(tablePath + (nonPartitionedTable ? "/*" : "/*/*/*/*")); - hoodieROViewDF.registerTempTable("hoodie_ro"); + snapshotQueryDF.registerTempTable("hoodie_ro"); spark.sql("describe hoodie_ro").show(); // all trips whose fare amount was greater than 2. spark.sql("select fare.amount, begin_lon, begin_lat, timestamp from hoodie_ro where fare.amount > 2.0").show(); @@ -219,15 +219,15 @@ public class HoodieJavaApp { /** * Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE */ - Dataset hoodieIncViewDF = spark.read().format("org.apache.hudi") - .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL()) + Dataset incQueryDF = spark.read().format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL()) // Only changes in write 2 above .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1) // For incremental view, pass in the root/base path of dataset .load(tablePath); LOG.info("You will only see records from : " + commitInstantTime2); - hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show(); + incQueryDF.groupBy(incQueryDF.col("_hoodie_commit_time")).count().show(); } } diff --git a/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark/src/test/java/HoodieJavaStreamingApp.java index f7921dab3..af55ea818 100644 --- a/hudi-spark/src/test/java/HoodieJavaStreamingApp.java +++ b/hudi-spark/src/test/java/HoodieJavaStreamingApp.java @@ -203,7 +203,7 @@ public class HoodieJavaStreamingApp { * Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE */ Dataset hoodieIncViewDF = spark.read().format("org.apache.hudi") - .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL()) + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL()) // Only changes in write 2 above .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1) // For incremental view, pass in the root/base path of dataset @@ -224,7 +224,7 @@ public class HoodieJavaStreamingApp { DataStreamWriter writer = streamingInput.writeStream().format("org.apache.hudi") .option("hoodie.insert.shuffle.parallelism", "2").option("hoodie.upsert.shuffle.parallelism", "2") - .option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType) .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") diff --git a/hudi-spark/src/test/scala/TestDataSource.scala b/hudi-spark/src/test/scala/TestDataSource.scala index 587c55af0..e9675ab29 100644 --- a/hudi-spark/src/test/scala/TestDataSource.scala +++ b/hudi-spark/src/test/scala/TestDataSource.scala @@ -116,7 +116,7 @@ class TestDataSource extends AssertionsForJUnit { // we have 2 commits, try pulling the first commit (which is not the latest) val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").get(0); val hoodieIncViewDF1 = spark.read.format("org.apache.hudi") - .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit) .load(basePath); @@ -127,7 +127,7 @@ class TestDataSource extends AssertionsForJUnit { // pull the latest commit val hoodieIncViewDF2 = spark.read.format("org.apache.hudi") - .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) .load(basePath); @@ -145,7 +145,7 @@ class TestDataSource extends AssertionsForJUnit { .options(commonOpts) .option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_STORAGE_TYPE_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) .mode(SaveMode.Overwrite) .save(basePath) @@ -193,7 +193,7 @@ class TestDataSource extends AssertionsForJUnit { assertEquals(hoodieROViewDF2.count(), totalUniqueKeyToGenerate) val hoodieIncViewDF2 = spark.read.format("org.apache.hudi") - .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) .load(basePath) assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt) @@ -263,7 +263,7 @@ class TestDataSource extends AssertionsForJUnit { // we have 2 commits, try pulling the first commit (which is not the latest) val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").get(0) val hoodieIncViewDF1 = spark.read.format("org.apache.hudi") - .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit) .load(destPath) @@ -275,7 +275,7 @@ class TestDataSource extends AssertionsForJUnit { // pull the latest commit val hoodieIncViewDF2 = spark.read.format("org.apache.hudi") - .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) .load(destPath) diff --git a/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite.scala b/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite.scala index be369712c..58ca9842f 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite.scala +++ b/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite.scala @@ -29,13 +29,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers { val originals = HoodieSparkSqlWriter.parametersWithWriteDefaults(Map.empty) val rhsKey = "hoodie.right.hand.side.key" val rhsVal = "hoodie.right.hand.side.val" - val modifier = Map(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL, STORAGE_TYPE_OPT_KEY -> MOR_STORAGE_TYPE_OPT_VAL, rhsKey -> rhsVal) + val modifier = Map(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL, TABLE_TYPE_OPT_KEY -> MOR_TABLE_TYPE_OPT_VAL, rhsKey -> rhsVal) val modified = HoodieSparkSqlWriter.parametersWithWriteDefaults(modifier) val matcher = (k: String, v: String) => modified(k) should be(v) originals foreach { case (OPERATION_OPT_KEY, _) => matcher(OPERATION_OPT_KEY, INSERT_OPERATION_OPT_VAL) - case (STORAGE_TYPE_OPT_KEY, _) => matcher(STORAGE_TYPE_OPT_KEY, MOR_STORAGE_TYPE_OPT_VAL) + case (TABLE_TYPE_OPT_KEY, _) => matcher(TABLE_TYPE_OPT_KEY, MOR_TABLE_TYPE_OPT_VAL) case (`rhsKey`, _) => matcher(rhsKey, rhsVal) case (k, v) => matcher(k, v) } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java index 1ce31af3e..a25b129ad 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java @@ -21,14 +21,14 @@ package org.apache.hudi.timeline.service; import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.SyncableFileSystemView; import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO; -import org.apache.hudi.common.table.timeline.dto.DataFileDTO; +import org.apache.hudi.common.table.timeline.dto.BaseFileDTO; import org.apache.hudi.common.table.timeline.dto.FileGroupDTO; import org.apache.hudi.common.table.timeline.dto.FileSliceDTO; import org.apache.hudi.common.table.timeline.dto.InstantDTO; import org.apache.hudi.common.table.timeline.dto.TimelineDTO; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView; -import org.apache.hudi.timeline.service.handlers.DataFileHandler; +import org.apache.hudi.timeline.service.handlers.BaseFileHandler; import org.apache.hudi.timeline.service.handlers.FileSliceHandler; import org.apache.hudi.timeline.service.handlers.TimelineHandler; @@ -60,14 +60,14 @@ public class FileSystemViewHandler { private final Javalin app; private final TimelineHandler instantHandler; private final FileSliceHandler sliceHandler; - private final DataFileHandler dataFileHandler; + private final BaseFileHandler dataFileHandler; public FileSystemViewHandler(Javalin app, Configuration conf, FileSystemViewManager viewManager) throws IOException { this.viewManager = viewManager; this.app = app; this.instantHandler = new TimelineHandler(conf, viewManager); this.sliceHandler = new FileSliceHandler(conf, viewManager); - this.dataFileHandler = new DataFileHandler(conf, viewManager); + this.dataFileHandler = new BaseFileHandler(conf, viewManager); } public void register() { @@ -160,14 +160,14 @@ public class FileSystemViewHandler { */ private void registerDataFilesAPI() { app.get(RemoteHoodieTableFileSystemView.LATEST_PARTITION_DATA_FILES_URL, new ViewHandler(ctx -> { - List dtos = dataFileHandler.getLatestDataFiles( + List dtos = dataFileHandler.getLatestDataFiles( ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow()); writeValueAsString(ctx, dtos); }, true)); app.get(RemoteHoodieTableFileSystemView.LATEST_PARTITION_DATA_FILE_URL, new ViewHandler(ctx -> { - List dtos = dataFileHandler.getLatestDataFile( + List dtos = dataFileHandler.getLatestDataFile( ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(), ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.FILEID_PARAM).getOrThrow()); @@ -175,13 +175,13 @@ public class FileSystemViewHandler { }, true)); app.get(RemoteHoodieTableFileSystemView.LATEST_ALL_DATA_FILES, new ViewHandler(ctx -> { - List dtos = dataFileHandler + List dtos = dataFileHandler .getLatestDataFiles(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow()); writeValueAsString(ctx, dtos); }, true)); app.get(RemoteHoodieTableFileSystemView.LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, new ViewHandler(ctx -> { - List dtos = dataFileHandler.getLatestDataFilesBeforeOrOn( + List dtos = dataFileHandler.getLatestDataFilesBeforeOrOn( ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(), ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.MAX_INSTANT_PARAM).getOrThrow()); @@ -189,7 +189,7 @@ public class FileSystemViewHandler { }, true)); app.get(RemoteHoodieTableFileSystemView.LATEST_DATA_FILE_ON_INSTANT_URL, new ViewHandler(ctx -> { - List dtos = dataFileHandler.getLatestDataFileOn( + List dtos = dataFileHandler.getLatestDataFileOn( ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(), ctx.queryParam(RemoteHoodieTableFileSystemView.INSTANT_PARAM), @@ -198,14 +198,14 @@ public class FileSystemViewHandler { }, true)); app.get(RemoteHoodieTableFileSystemView.ALL_DATA_FILES, new ViewHandler(ctx -> { - List dtos = dataFileHandler.getAllDataFiles( + List dtos = dataFileHandler.getAllDataFiles( ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow()); writeValueAsString(ctx, dtos); }, true)); app.get(RemoteHoodieTableFileSystemView.LATEST_DATA_FILES_RANGE_INSTANT_URL, new ViewHandler(ctx -> { - List dtos = dataFileHandler.getLatestDataFilesInRange( + List dtos = dataFileHandler.getLatestDataFilesInRange( ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), Arrays .asList(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.INSTANTS_PARAM).getOrThrow().split(","))); writeValueAsString(ctx, dtos); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/DataFileHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java similarity index 51% rename from hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/DataFileHandler.java rename to hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java index 39a667ad1..8724f28d6 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/DataFileHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java @@ -18,7 +18,7 @@ package org.apache.hudi.timeline.service.handlers; -import org.apache.hudi.common.table.timeline.dto.DataFileDTO; +import org.apache.hudi.common.table.timeline.dto.BaseFileDTO; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hadoop.conf.Configuration; @@ -30,49 +30,49 @@ import java.util.List; import java.util.stream.Collectors; /** - * REST Handler servicing data-file requests. + * REST Handler servicing base-file requests. */ -public class DataFileHandler extends Handler { +public class BaseFileHandler extends Handler { - public DataFileHandler(Configuration conf, FileSystemViewManager viewManager) throws IOException { + public BaseFileHandler(Configuration conf, FileSystemViewManager viewManager) throws IOException { super(conf, viewManager); } - public List getLatestDataFiles(String basePath, String partitionPath) { - return viewManager.getFileSystemView(basePath).getLatestDataFiles(partitionPath) - .map(DataFileDTO::fromHoodieDataFile).collect(Collectors.toList()); + public List getLatestDataFiles(String basePath, String partitionPath) { + return viewManager.getFileSystemView(basePath).getLatestBaseFiles(partitionPath) + .map(BaseFileDTO::fromHoodieBaseFile).collect(Collectors.toList()); } - public List getLatestDataFile(String basePath, String partitionPath, String fileId) { - return viewManager.getFileSystemView(basePath).getLatestDataFile(partitionPath, fileId) - .map(DataFileDTO::fromHoodieDataFile).map(Arrays::asList).orElse(new ArrayList<>()); + public List getLatestDataFile(String basePath, String partitionPath, String fileId) { + return viewManager.getFileSystemView(basePath).getLatestBaseFile(partitionPath, fileId) + .map(BaseFileDTO::fromHoodieBaseFile).map(Arrays::asList).orElse(new ArrayList<>()); } - public List getLatestDataFiles(String basePath) { - return viewManager.getFileSystemView(basePath).getLatestDataFiles().map(DataFileDTO::fromHoodieDataFile) + public List getLatestDataFiles(String basePath) { + return viewManager.getFileSystemView(basePath).getLatestBaseFiles().map(BaseFileDTO::fromHoodieBaseFile) .collect(Collectors.toList()); } - public List getLatestDataFilesBeforeOrOn(String basePath, String partitionPath, String maxInstantTime) { - return viewManager.getFileSystemView(basePath).getLatestDataFilesBeforeOrOn(partitionPath, maxInstantTime) - .map(DataFileDTO::fromHoodieDataFile).collect(Collectors.toList()); + public List getLatestDataFilesBeforeOrOn(String basePath, String partitionPath, String maxInstantTime) { + return viewManager.getFileSystemView(basePath).getLatestBaseFilesBeforeOrOn(partitionPath, maxInstantTime) + .map(BaseFileDTO::fromHoodieBaseFile).collect(Collectors.toList()); } - public List getLatestDataFileOn(String basePath, String partitionPath, String instantTime, - String fileId) { - List result = new ArrayList<>(); - viewManager.getFileSystemView(basePath).getDataFileOn(partitionPath, instantTime, fileId) - .map(DataFileDTO::fromHoodieDataFile).ifPresent(result::add); + public List getLatestDataFileOn(String basePath, String partitionPath, String instantTime, + String fileId) { + List result = new ArrayList<>(); + viewManager.getFileSystemView(basePath).getBaseFileOn(partitionPath, instantTime, fileId) + .map(BaseFileDTO::fromHoodieBaseFile).ifPresent(result::add); return result; } - public List getLatestDataFilesInRange(String basePath, List instants) { - return viewManager.getFileSystemView(basePath).getLatestDataFilesInRange(instants) - .map(DataFileDTO::fromHoodieDataFile).collect(Collectors.toList()); + public List getLatestDataFilesInRange(String basePath, List instants) { + return viewManager.getFileSystemView(basePath).getLatestBaseFilesInRange(instants) + .map(BaseFileDTO::fromHoodieBaseFile).collect(Collectors.toList()); } - public List getAllDataFiles(String basePath, String partitionPath) { - return viewManager.getFileSystemView(basePath).getAllDataFiles(partitionPath).map(DataFileDTO::fromHoodieDataFile) + public List getAllDataFiles(String basePath, String partitionPath) { + return viewManager.getFileSystemView(basePath).getAllBaseFiles(partitionPath).map(BaseFileDTO::fromHoodieBaseFile) .collect(Collectors.toList()); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java index 522d39a18..7d944d355 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java @@ -19,12 +19,12 @@ package org.apache.hudi.utilities; import org.apache.hudi.common.SerializableConfiguration; -import org.apache.hudi.common.model.HoodieDataFile; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; -import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView; +import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.FSUtils; @@ -73,7 +73,7 @@ public class HoodieSnapshotCopier implements Serializable { FileSystem fs = FSUtils.getFs(baseDir, jsc.hadoopConfiguration()); final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration()); final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir); - final ReadOptimizedView fsView = new HoodieTableFileSystemView(tableMetadata, + final BaseFileOnlyView fsView = new HoodieTableFileSystemView(tableMetadata, tableMetadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()); // Get the latest commit Option latestCommit = @@ -101,7 +101,7 @@ public class HoodieSnapshotCopier implements Serializable { // Only take latest version files <= latestCommit. FileSystem fs1 = FSUtils.getFs(baseDir, serConf.newCopy()); List> filePaths = new ArrayList<>(); - Stream dataFiles = fsView.getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp); + Stream dataFiles = fsView.getLatestBaseFilesBeforeOrOn(partition, latestCommitTimestamp); dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()))); // also need to copy over partition metadata diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 6ba14877a..5f3259acf 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -208,7 +208,7 @@ public class DeltaSync implements Serializable { } else { this.commitTimelineOpt = Option.empty(); HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, - cfg.storageType, cfg.targetTableName, "archived", cfg.payloadClassName); + cfg.tableType, cfg.targetTableName, "archived", cfg.payloadClassName); } } @@ -270,7 +270,7 @@ public class DeltaSync implements Serializable { } } else { HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, - cfg.storageType, cfg.targetTableName, "archived", cfg.payloadClassName); + cfg.tableType, cfg.targetTableName, "archived", cfg.payloadClassName); } if (!resumeCheckpointStr.isPresent() && cfg.checkpoint != null) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java index a56591df0..b8ff404c8 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -156,13 +156,11 @@ public class HoodieDeltaStreamer implements Serializable { required = true) public String targetBasePath; - // TODO: How to obtain hive configs to register? @Parameter(names = {"--target-table"}, description = "name of the target table in Hive", required = true) public String targetTableName; - @Parameter(names = {"--storage-type"}, description = "Type of Storage. COPY_ON_WRITE (or) MERGE_ON_READ", - required = true) - public String storageType; + @Parameter(names = {"--table-type"}, description = "Type of table. COPY_ON_WRITE (or) MERGE_ON_READ", required = true) + public String tableType; @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are " @@ -273,12 +271,12 @@ public class HoodieDeltaStreamer implements Serializable { public boolean isAsyncCompactionEnabled() { return continuousMode && !forceDisableCompaction - && HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(storageType)); + && HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType)); } public boolean isInlineCompactionEnabled() { return !continuousMode && !forceDisableCompaction - && HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(storageType)); + && HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType)); } } @@ -356,10 +354,10 @@ public class HoodieDeltaStreamer implements Serializable { new HoodieTableMetaClient(new Configuration(fs.getConf()), cfg.targetBasePath, false); tableType = meta.getTableType(); // This will guarantee there is no surprise with table type - Preconditions.checkArgument(tableType.equals(HoodieTableType.valueOf(cfg.storageType)), - "Hoodie table is of type " + tableType + " but passed in CLI argument is " + cfg.storageType); + Preconditions.checkArgument(tableType.equals(HoodieTableType.valueOf(cfg.tableType)), + "Hoodie table is of type " + tableType + " but passed in CLI argument is " + cfg.tableType); } else { - tableType = HoodieTableType.valueOf(cfg.storageType); + tableType = HoodieTableType.valueOf(cfg.tableType); } this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig(); @@ -502,7 +500,6 @@ public class HoodieDeltaStreamer implements Serializable { public AsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client) { this.jssc = jssc; this.compactor = new Compactor(client, jssc); - // TODO: HUDI-157 : Only allow 1 compactor to run in parallel till Incremental View on MOR is fully implemented. this.maxConcurrentCompaction = 1; } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java index 09c4da021..c0059ad45 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java @@ -71,7 +71,7 @@ public class SchedulerConfGenerator { Map additionalSparkConfigs = new HashMap<>(); if (sparkSchedulerMode.isPresent() && "FAIR".equals(sparkSchedulerMode.get()) && cfg.continuousMode - && cfg.storageType.equals(HoodieTableType.MERGE_ON_READ.name())) { + && cfg.tableType.equals(HoodieTableType.MERGE_ON_READ.name())) { String sparkSchedulingConfFile = generateAndStoreConfig(cfg.deltaSyncSchedulingWeight, cfg.compactSchedulingWeight, cfg.deltaSyncSchedulingMinShare, cfg.compactSchedulingMinShare); additionalSparkConfigs.put(SPARK_SCHEDULER_ALLOCATION_FILE_KEY, sparkSchedulingConfFile); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java index 888eec702..a31683b63 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java @@ -115,7 +115,7 @@ public class HoodieIncrSource extends RowSource { // Do Incr pull. Set end instant if available DataFrameReader reader = sparkSession.read().format("org.apache.hudi") - .option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL()) + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL()) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), instantEndpts.getLeft()) .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY(), instantEndpts.getRight()); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java index d2ff3033c..641c47bec 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java @@ -185,11 +185,11 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op, String transformerClassName, String propsFilename, boolean enableHiveSync, boolean useSchemaProviderClass, boolean updatePayloadClass, - String payloadClassName, String storageType) { + String payloadClassName, String tableType) { HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config(); cfg.targetBasePath = basePath; cfg.targetTableName = "hoodie_trips"; - cfg.storageType = storageType == null ? "COPY_ON_WRITE" : storageType; + cfg.tableType = tableType == null ? "COPY_ON_WRITE" : tableType; cfg.sourceClassName = TestDataSource.class.getName(); cfg.transformerClassName = transformerClassName; cfg.operation = op; @@ -211,7 +211,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config(); cfg.targetBasePath = basePath; cfg.targetTableName = "hoodie_trips_copy"; - cfg.storageType = "COPY_ON_WRITE"; + cfg.tableType = "COPY_ON_WRITE"; cfg.sourceClassName = HoodieIncrSource.class.getName(); cfg.operation = op; cfg.sourceOrderingField = "timestamp"; @@ -386,7 +386,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, Operation.UPSERT); cfg.continuousMode = true; - cfg.storageType = tableType.name(); + cfg.tableType = tableType.name(); cfg.configs.add(String.format("%s=%d", TestSourceConfig.MAX_UNIQUE_RECORDS_PROP, totalRecords)); cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); @@ -489,11 +489,11 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { // Test Hive integration HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(), dfs); - assertTrue("Table " + hiveSyncConfig.tableName + " should exist", hiveClient.doesTableExist()); + assertTrue("Table " + hiveSyncConfig.tableName + " should exist", hiveClient.doesTableExist(hiveSyncConfig.tableName)); assertEquals("Table partitions should match the number of partitions we wrote", 1, - hiveClient.scanTablePartitions().size()); + hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size()); assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", lastInstantForUpstreamTable, - hiveClient.getLastCommitTimeSynced().get()); + hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get()); } @Test diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchedulerConfGenerator.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchedulerConfGenerator.java index af1f566c2..754f469a4 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchedulerConfGenerator.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchedulerConfGenerator.java @@ -43,12 +43,12 @@ public class TestSchedulerConfGenerator { assertNull("continuousMode is false", configs.get(SchedulerConfGenerator.SPARK_SCHEDULER_ALLOCATION_FILE_KEY)); cfg.continuousMode = true; - cfg.storageType = HoodieTableType.COPY_ON_WRITE.name(); + cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); configs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg); - assertNull("storageType is not MERGE_ON_READ", + assertNull("table type is not MERGE_ON_READ", configs.get(SchedulerConfGenerator.SPARK_SCHEDULER_ALLOCATION_FILE_KEY)); - cfg.storageType = HoodieTableType.MERGE_ON_READ.name(); + cfg.tableType = HoodieTableType.MERGE_ON_READ.name(); configs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg); assertNotNull("all satisfies", configs.get(SchedulerConfGenerator.SPARK_SCHEDULER_ALLOCATION_FILE_KEY)); }