diff --git a/LICENSE b/LICENSE index dc4edaaf33421..1e2174731f161 100644 --- a/LICENSE +++ b/LICENSE @@ -206,6 +206,7 @@ This product includes code from Apache Hive. * org.apache.hadoop.hive.ql.io.CombineHiveInputFormat copied to org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat +* org.apache.hadoop.hive.serde2.ColumnProjectionUtils copied and modified to org.apache.hudi.hadoop.HoodieColumnProjectionUtils Copyright: 2011-2019 The Apache Software Foundation Home page: http://hive.apache.org/ diff --git a/docker/demo/compaction.commands b/docker/demo/compaction.commands index 9bb8eb82aa785..6abdad743e57c 100644 --- a/docker/demo/compaction.commands +++ b/docker/demo/compaction.commands @@ -19,4 +19,7 @@ connect --path /user/hive/warehouse/stock_ticks_mor compactions show all compaction schedule --hoodieConfigs hoodie.compact.inline.max.delta.commits=1 compaction run --parallelism 2 --sparkMemory 1G --schemaFilePath /var/demo/config/schema.avsc --retry 1 - +connect --path /user/hive/warehouse/stock_ticks_mor_bs +compactions show all +compaction schedule --hoodieConfigs hoodie.compact.inline.max.delta.commits=1 +compaction run --parallelism 2 --sparkMemory 1G --schemaFilePath /var/demo/config/schema.avsc --retry 1 diff --git a/docker/demo/hive-batch1.commands b/docker/demo/hive-batch1.commands index 93bf3b67930aa..021c6d55b800d 100644 --- a/docker/demo/hive-batch1.commands +++ b/docker/demo/hive-batch1.commands @@ -25,4 +25,12 @@ select symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GO 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'; +select symbol, max(ts) from stock_ticks_cow_bs group by symbol HAVING symbol = 'GOOG'; +select symbol, max(ts) from stock_ticks_mor_bs_ro group by symbol HAVING symbol = 'GOOG'; +select symbol, max(ts) from stock_ticks_mor_bs_rt group by symbol HAVING symbol = 'GOOG'; + +select symbol, ts, volume, open, close from stock_ticks_cow_bs where symbol = 'GOOG'; +select symbol, ts, volume, open, close from stock_ticks_mor_bs_ro where symbol = 'GOOG'; +select symbol, ts, volume, open, close from stock_ticks_mor_bs_rt where symbol = 'GOOG'; + !quit diff --git a/docker/demo/hive-batch2-after-compaction.commands b/docker/demo/hive-batch2-after-compaction.commands index 6b087019d5cca..06582a309ae00 100644 --- a/docker/demo/hive-batch2-after-compaction.commands +++ b/docker/demo/hive-batch2-after-compaction.commands @@ -23,4 +23,10 @@ select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = ' 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'; +select symbol, max(ts) from stock_ticks_mor_bs_ro group by symbol HAVING symbol = 'GOOG'; +select symbol, max(ts) from stock_ticks_mor_bs_rt group by symbol HAVING symbol = 'GOOG'; + +select symbol, ts, volume, open, close from stock_ticks_mor_bs_ro where symbol = 'GOOG'; +select symbol, ts, volume, open, close from stock_ticks_mor_bs_rt where symbol = 'GOOG'; + !quit diff --git a/docker/demo/hive-incremental-cow.commands b/docker/demo/hive-incremental-cow.commands index 7f43548071863..702b2afa52733 100644 --- a/docker/demo/hive-incremental-cow.commands +++ b/docker/demo/hive-incremental-cow.commands @@ -23,5 +23,11 @@ set hoodie.stock_ticks_cow.consume.start.timestamp='${min.commit.time}'; select symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG' and `_hoodie_commit_time` > '${min.commit.time}'; +set hoodie.stock_ticks_cow_bs.consume.mode=INCREMENTAL; +set hoodie.stock_ticks_cow_bs.consume.max.commits=3; +set hoodie.stock_ticks_cow_bs.consume.start.timestamp='00000000000001'; + +select symbol, ts, volume, open, close from stock_ticks_cow_bs where symbol = 'GOOG' and `_hoodie_commit_time` > '00000000000001'; + !quit diff --git a/docker/demo/hive-incremental-mor-ro.commands b/docker/demo/hive-incremental-mor-ro.commands index 8b97c0aac9b5e..51683c010a496 100644 --- a/docker/demo/hive-incremental-mor-ro.commands +++ b/docker/demo/hive-incremental-mor-ro.commands @@ -23,5 +23,11 @@ set hoodie.stock_ticks_mor.consume.start.timestamp='${min.commit.time}'; select symbol, ts, volume, open, close from stock_ticks_mor_ro where symbol = 'GOOG' and `_hoodie_commit_time` > '${min.commit.time}'; +set hoodie.stock_ticks_mor_bs.consume.mode=INCREMENTAL; +set hoodie.stock_ticks_mor_bs.consume.max.commits=3; +set hoodie.stock_ticks_mor_bs.consume.start.timestamp='00000000000001'; + +select symbol, ts, volume, open, close from stock_ticks_mor_bs_ro where symbol = 'GOOG' and `_hoodie_commit_time` > '00000000000001'; + !quit diff --git a/docker/demo/hive-incremental-mor-rt.commands b/docker/demo/hive-incremental-mor-rt.commands index a81fb77e077d8..c29fc7ce55730 100644 --- a/docker/demo/hive-incremental-mor-rt.commands +++ b/docker/demo/hive-incremental-mor-rt.commands @@ -23,5 +23,11 @@ set hoodie.stock_ticks_mor.consume.start.timestamp='${min.commit.time}'; select symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG' and `_hoodie_commit_time` > '${min.commit.time}'; +set hoodie.stock_ticks_mor_bs.consume.mode=INCREMENTAL; +set hoodie.stock_ticks_mor_bs.consume.max.commits=3; +set hoodie.stock_ticks_mor_bs.consume.start.timestamp='00000000000001'; + +select symbol, ts, volume, open, close from stock_ticks_mor_bs_rt where symbol = 'GOOG' and `_hoodie_commit_time` > '00000000000001'; + !quit diff --git a/docker/demo/sparksql-batch1.commands b/docker/demo/sparksql-batch1.commands index 727aa1633154d..4de2486c6ce58 100644 --- a/docker/demo/sparksql-batch1.commands +++ b/docker/demo/sparksql-batch1.commands @@ -27,4 +27,14 @@ spark.sql("select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING 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) +// Bootstrapped Copy-On-Write table +spark.sql("select symbol, max(ts) from stock_ticks_cow_bs group by symbol HAVING symbol = 'GOOG'").show(100, false) +spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow_bs where symbol = 'GOOG'").show(100, false) + +// Bootstrapped Merge-On-Read table +spark.sql("select symbol, max(ts) from stock_ticks_mor_bs_ro group by symbol HAVING symbol = 'GOOG'").show(100, false) +spark.sql("select symbol, max(ts) from stock_ticks_mor_bs_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_bs_ro where symbol = 'GOOG'").show(100, false) +spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_bs_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 391e11b971a27..739d991dbbc1d 100644 --- a/docker/demo/sparksql-batch2.commands +++ b/docker/demo/sparksql-batch2.commands @@ -26,4 +26,14 @@ spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from s 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) + // Copy-On-Write Bootstrapped table +spark.sql("select symbol, max(ts) from stock_ticks_cow_bs group by symbol HAVING symbol = 'GOOG'").show(100, false) +spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow_bs where symbol = 'GOOG'").show(100, false) + +// Merge-On-Read table Bootstrapped Table +spark.sql("select symbol, max(ts) from stock_ticks_mor_bs_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_bs_ro where symbol = 'GOOG'").show(100, false) +spark.sql("select symbol, max(ts) from stock_ticks_mor_bs_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_bs_rt where symbol = 'GOOG'").show(100, false) + System.exit(0) diff --git a/docker/demo/sparksql-bootstrap-prep-source.commands b/docker/demo/sparksql-bootstrap-prep-source.commands new file mode 100644 index 0000000000000..23db3e4d38c4b --- /dev/null +++ b/docker/demo/sparksql-bootstrap-prep-source.commands @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +import org.apache.spark.sql.functions.col + +val df = spark.read.format("org.apache.hudi").load("/user/hive/warehouse/stock_ticks_cow/*/*/*").drop("_hoodie_commit_time", "_hoodie_record_key", "_hoodie_file_name", "_hoodie_commit_seqno", "_hoodie_partition_path") +df.write.format("parquet").save("/user/hive/warehouse/stock_ticks_cow_bs_src/2018/08/31/") +System.exit(0) diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands index 8e3e153e27e7e..febfcd28a1116 100644 --- a/docker/demo/sparksql-incremental.commands +++ b/docker/demo/sparksql-incremental.commands @@ -52,8 +52,38 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl mode(SaveMode.Overwrite). save("/user/hive/warehouse/stock_ticks_derived_mor"); -spark.sql("show tables").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 +val hoodieIncQueryBsDF = spark.read.format("org.apache.hudi"). + option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL). + option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "00000000000001"). + load("/user/hive/warehouse/stock_ticks_cow_bs"); +hoodieIncQueryBsDF.registerTempTable("stock_ticks_cow_bs_incr") +spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_cow_bs_incr where symbol = 'GOOG'").show(100, false); + +spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, close from stock_ticks_cow_bs_incr"). + write.format("org.apache.hudi"). + option("hoodie.insert.shuffle.parallelism", "2"). + option("hoodie.upsert.shuffle.parallelism","2"). + 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"). + option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "ts"). + option(HoodieWriteConfig.TABLE_NAME, "stock_ticks_derived_mor_bs"). + option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY, "stock_ticks_derived_mor_bs"). + option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY, "default"). + option(DataSourceWriteOptions.HIVE_URL_OPT_KEY, "jdbc:hive2://hiveserver:10000"). + option(DataSourceWriteOptions.HIVE_USER_OPT_KEY, "hive"). + option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY, "hive"). + option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY, "true"). + option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY, "datestr"). + mode(SaveMode.Overwrite). + save("/user/hive/warehouse/stock_ticks_derived_mor_bs"); + +spark.sql("show tables").show(20, false) +spark.sql("select count(*) from stock_ticks_derived_mor_bs_ro").show(20, false) +spark.sql("select count(*) from stock_ticks_derived_mor_bs_rt").show(20, false) + +System.exit(0); diff --git a/hudi-cli/hudi-cli.sh b/hudi-cli/hudi-cli.sh index b6e708c14436d..bbfba85a8010e 100755 --- a/hudi-cli/hudi-cli.sh +++ b/hudi-cli/hudi-cli.sh @@ -25,4 +25,7 @@ if [ -z "$CLIENT_JAR" ]; then echo "Client jar location not set, please set it in conf/hudi-env.sh" fi -java -cp ${HADOOP_CONF_DIR}:${SPARK_CONF_DIR}:$DIR/target/lib/*:$HOODIE_JAR:${CLIENT_JAR} -DSPARK_CONF_DIR=${SPARK_CONF_DIR} -DHADOOP_CONF_DIR=${HADOOP_CONF_DIR} org.springframework.shell.Bootstrap $@ +OTHER_JARS=`ls ${DIR}/target/lib/* | grep -v 'hudi-[^/]*jar' | tr '\n' ':'` + +echo "Running : java -cp ${HADOOP_CONF_DIR}:${SPARK_CONF_DIR}:${HOODIE_JAR}:${OTHER_JARS}:${CLIENT_JAR} -DSPARK_CONF_DIR=${SPARK_CONF_DIR} -DHADOOP_CONF_DIR=${HADOOP_CONF_DIR} org.springframework.shell.Bootstrap $@" +java -cp ${HADOOP_CONF_DIR}:${SPARK_CONF_DIR}:${HOODIE_JAR}:${OTHER_JARS}:${CLIENT_JAR} -DSPARK_CONF_DIR=${SPARK_CONF_DIR} -DHADOOP_CONF_DIR=${HADOOP_CONF_DIR} org.springframework.shell.Bootstrap $@ diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml index 388e4145a18e2..ec15cce76b3fa 100644 --- a/hudi-cli/pom.xml +++ b/hudi-cli/pom.xml @@ -139,26 +139,6 @@ - - org.apache.hudi - hudi-client - ${project.version} - - - org.apache.hudi - hudi-common - ${project.version} - - - org.apache.hudi - hudi-hive-sync - ${project.version} - - - org.apache.hudi - hudi-utilities_${scala.binary.version} - ${project.version} - org.apache.hudi hudi-common @@ -198,6 +178,12 @@ + + org.apache.hudi + hudi-utilities-bundle_${scala.binary.version} + ${project.version} + + log4j diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java index 9b55fe26eaeed..ffbf70e12a9bc 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java @@ -213,7 +213,7 @@ public String scheduleCompact(@CliOption(key = "sparkMemory", unspecifiedDefault if (exitCode != 0) { return "Failed to run compaction for " + compactionInstantTime; } - return "Compaction successfully completed for " + compactionInstantTime; + return "Attempted to schedule compaction for " + compactionInstantTime; } @CliCommand(value = "compaction run", help = "Run Compaction for given instant time") 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 2e32515cffe09..ef76ee4e2f1ad 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 @@ -22,6 +22,7 @@ import org.apache.hudi.cli.HoodiePrintHelper; import org.apache.hudi.cli.HoodieTableHeaderFields; import org.apache.hudi.cli.TableHeader; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -239,7 +240,7 @@ private HoodieTableFileSystemView buildFileSystemView(String globRegex, String m new HoodieTableMetaClient(client.getHadoopConf(), client.getBasePath(), true); FileSystem fs = HoodieCLI.fs; String globPath = String.format("%s/%s/*", client.getBasePath(), globRegex); - FileStatus[] statuses = fs.globStatus(new Path(globPath)); + List statuses = FSUtils.getGlobStatusExcludingMetaFolder(fs, new Path(globPath)); Stream instantsStream; HoodieTimeline timeline; @@ -269,6 +270,6 @@ private HoodieTableFileSystemView buildFileSystemView(String globRegex, String m HoodieTimeline filteredTimeline = new HoodieDefaultTimeline(instantsStream, (Function> & Serializable) metaClient.getActiveTimeline()::getInstantDetails); - return new HoodieTableFileSystemView(metaClient, filteredTimeline, statuses); + return new HoodieTableFileSystemView(metaClient, filteredTimeline, statuses.toArray(new FileStatus[0])); } } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java index a5fe4fe53826f..f8e82ae618581 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java @@ -22,6 +22,7 @@ import org.apache.hudi.cli.HoodiePrintHelper; import org.apache.hudi.cli.HoodieTableHeaderFields; import org.apache.hudi.cli.TableHeader; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -53,7 +54,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; import java.util.List; @@ -82,7 +82,7 @@ public String showLogFileCommits( throws IOException { FileSystem fs = HoodieCLI.getTableMetaClient().getFs(); - List logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern))) + List logFilePaths = FSUtils.getGlobStatusExcludingMetaFolder(fs, new Path(logFilePathPattern)).stream() .map(status -> status.getPath().toString()).collect(Collectors.toList()); Map, Map>, Integer>>> commitCountAndMetadata = new HashMap<>(); @@ -175,7 +175,7 @@ public String showLogFileRecords( HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); FileSystem fs = client.getFs(); - List logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern))) + List logFilePaths = FSUtils.getGlobStatusExcludingMetaFolder(fs, new Path(logFilePathPattern)).stream() .map(status -> status.getPath().toString()).sorted(Comparator.reverseOrder()) .collect(Collectors.toList()); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java index 72cf6c02da3b4..66c5563102848 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/StatsCommand.java @@ -118,7 +118,7 @@ public String fileSizeStats( FileSystem fs = HoodieCLI.fs; String globPath = String.format("%s/%s/*", HoodieCLI.getTableMetaClient().getBasePath(), globRegex); - FileStatus[] statuses = fs.globStatus(new Path(globPath)); + List statuses = FSUtils.getGlobStatusExcludingMetaFolder(fs, new Path(globPath)); // max, min, #small files < 10MB, 50th, avg, 95th Histogram globalHistogram = new Histogram(new UniformReservoir(MAX_FILES)); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java index 78ae35e198b1c..9c947e4d407e3 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java @@ -88,8 +88,7 @@ public String createTable( @CliOption(key = {"archiveLogFolder"}, help = "Folder Name for storing archived timeline") String archiveFolder, @CliOption(key = {"layoutVersion"}, help = "Specific Layout Version to use") Integer layoutVersion, @CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "org.apache.hudi.common.model.HoodieAvroPayload", - help = "Payload Class") final String payloadClass) - throws IOException { + help = "Payload Class") final String payloadClass) throws IOException { boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java index 4c7ce8819d534..7d5cee6939e86 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java @@ -62,6 +62,7 @@ public void init() throws IOException { // Create table and connect String tableName = "test_table"; tablePath = basePath + File.separator + tableName; + new TableCommand().createTable( tablePath, tableName, "COPY_ON_WRITE", "", 1, "org.apache.hudi.common.model.HoodieAvroPayload"); diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java b/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java index 9782b46b6ab12..2a1520a77fd8f 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java +++ b/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java @@ -149,6 +149,35 @@ public JavaRDD> filterExists(JavaRDD> hoodieReco return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); } + /** + * Main API to run bootstrap to hudi. + */ + public void bootstrap(Option> extraMetadata) { + if (rollbackPending) { + rollBackInflightBootstrap(); + } + HoodieTable table = getTableAndInitCtx(WriteOperationType.UPSERT); + table.bootstrap(jsc, extraMetadata); + } + + /** + * Main API to rollback pending bootstrap. + */ + protected void rollBackInflightBootstrap() { + LOG.info("Rolling back pending bootstrap if present"); + HoodieTable table = HoodieTable.create(config, hadoopConf); + HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); + Option instant = Option.fromJavaOptional( + inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst()); + if (instant.isPresent() && HoodieTimeline.compareTimestamps(instant.get(), HoodieTimeline.LESSER_THAN_OR_EQUALS, + HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) { + LOG.info("Found pending bootstrap instants. Rolling them back"); + table.rollbackBootstrap(jsc, HoodieActiveTimeline.createNewInstantTime()); + LOG.info("Finished rolling back pending bootstrap"); + } + + } + /** * Upsert a batch of new records into Hoodie table at the supplied instantTime. * @@ -671,7 +700,13 @@ private void rollbackPendingCommits() { List commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); for (String commit : commits) { - rollback(commit); + if (HoodieTimeline.compareTimestamps(commit, HoodieTimeline.LESSER_THAN_OR_EQUALS, + HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) { + rollBackInflightBootstrap(); + break; + } else { + rollback(commit); + } } } diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java new file mode 100644 index 0000000000000..769f3c25c8948 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap; + +/** + * Identifies different types of bootstrap. + */ +public enum BootstrapMode { + /** + * In this mode, record level metadata is generated for each source record and both original record and metadata + * for each record copied. + */ + FULL_RECORD, + + /** + * In this mode, record level metadata alone is generated for each source record and stored in new bootstrap location. + */ + METADATA_ONLY +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java new file mode 100644 index 0000000000000..fa508e42f120c --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap; + +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; + +public class BootstrapRecordPayload implements HoodieRecordPayload { + + private final GenericRecord record; + + public BootstrapRecordPayload(GenericRecord record) { + this.record = record; + } + + @Override + public BootstrapRecordPayload preCombine(BootstrapRecordPayload another) { + return this; + } + + @Override + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) { + return Option.ofNullable(record); + } + + @Override + public Option getInsertValue(Schema schema) { + return Option.ofNullable(record); + } + +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapSchemaProvider.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapSchemaProvider.java new file mode 100644 index 0000000000000..61e29c2cfab68 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapSchemaProvider.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap; + +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.util.ParquetUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.spark.api.java.JavaSparkContext; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; + +import java.util.List; + +/** + * Bootstrap Schema Provider. Schema provided in config is used. If not available, use schema from Parquet + */ +public class BootstrapSchemaProvider { + + protected final HoodieWriteConfig writeConfig; + + public BootstrapSchemaProvider(HoodieWriteConfig writeConfig) { + this.writeConfig = writeConfig; + } + + /** + * Main API to select avro schema for bootstrapping. + * @param jsc Java Spark Context + * @param partitions List of partitions with files within them + * @return Avro Schema + */ + public final Schema getBootstrapSchema(JavaSparkContext jsc, List>> partitions) { + if (writeConfig.getSchema() != null) { + // Use schema specified by user if set + return Schema.parse(writeConfig.getSchema()); + } + return getBootstrapSourceSchema(jsc, partitions); + } + + /** + * Select a random file to be used to generate avro schema. + * Override this method to get custom schema selection. + * @param jsc Java Spark Context + * @param partitions List of partitions with files within them + * @return Avro Schema + */ + protected Schema getBootstrapSourceSchema(JavaSparkContext jsc, + List>> partitions) { + return partitions.stream().flatMap(p -> p.getValue().stream()) + .map(fs -> { + try { + Path filePath = FileStatusUtils.toPath(fs.getPath()); + return ParquetUtils.readAvroSchema(jsc.hadoopConfiguration(), filePath); + } catch (Exception ex) { + return null; + } + }).filter(x -> x != null).findAny().get(); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java new file mode 100644 index 0000000000000..29d0646a4083b --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.BootstrapFileMapping; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.util.collection.Pair; + +/** + * WriteStatus for Bootstrap. + */ +public class BootstrapWriteStatus extends WriteStatus { + + private BootstrapFileMapping sourceFileMapping; + + public BootstrapWriteStatus(Boolean trackSuccessRecords, Double failureFraction) { + super(trackSuccessRecords, failureFraction); + } + + public BootstrapFileMapping getBootstrapSourceFileMapping() { + return sourceFileMapping; + } + + public Pair getBootstrapSourceAndWriteStat() { + return Pair.of(getBootstrapSourceFileMapping(), getStat()); + } + + public void setBootstrapSourceFileMapping(BootstrapFileMapping sourceFileMapping) { + this.sourceFileMapping = sourceFileMapping; + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java new file mode 100644 index 0000000000000..8b077ad9f0e31 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap; + +import java.io.Serializable; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.List; + +/** + * Creates RDD of Hoodie Records with complete record data, given a list of partitions to be bootstrapped. + */ +public abstract class FullRecordBootstrapDataProvider implements Serializable { + + protected static final Logger LOG = LogManager.getLogger(FullRecordBootstrapDataProvider.class); + + protected final TypedProperties props; + protected final transient JavaSparkContext jsc; + + public FullRecordBootstrapDataProvider(TypedProperties props, JavaSparkContext jsc) { + this.props = props; + this.jsc = jsc; + } + + /** + * Generates a list of input partition and files and returns a RDD representing source. + * @param tableName Hudi Table Name + * @param sourceBasePath Source Base Path + * @param partitionPaths Partition Paths + * @return JavaRDD of input records + */ + public abstract JavaRDD generateInputRecordRDD(String tableName, + String sourceBasePath, List>> partitionPaths); +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java new file mode 100644 index 0000000000000..4e098c6b3d413 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap.selector; + +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +/** + * Pluggable Partition Selector for selecting partitions to perform full or metadata-only bootstrapping. + */ +public abstract class BootstrapModeSelector implements Serializable { + + protected final HoodieWriteConfig writeConfig; + + public BootstrapModeSelector(HoodieWriteConfig writeConfig) { + this.writeConfig = writeConfig; + } + + /** + * Classify partitions for the purpose of bootstrapping. For a non-partitioned source, input list will be one entry. + * + * @param partitions List of partitions with files present in each partitions + * @return a partitions grouped by bootstrap mode + */ + public abstract Map> select(List>> partitions); +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java new file mode 100644 index 0000000000000..43fae7000f456 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap.selector; + +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +public class BootstrapRegexModeSelector extends BootstrapModeSelector { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = LogManager.getLogger(BootstrapRegexModeSelector.class); + + private final Pattern pattern; + private final BootstrapMode bootstrapModeOnMatch; + private final BootstrapMode defaultMode; + + public BootstrapRegexModeSelector(HoodieWriteConfig writeConfig) { + super(writeConfig); + this.pattern = Pattern.compile(writeConfig.getBootstrapModeSelectorRegex()); + this.bootstrapModeOnMatch = writeConfig.getBootstrapModeForRegexMatch(); + this.defaultMode = BootstrapMode.FULL_RECORD.equals(bootstrapModeOnMatch) + ? BootstrapMode.METADATA_ONLY : BootstrapMode.FULL_RECORD; + LOG.info("Default Mode :" + defaultMode + ", on Match Mode :" + bootstrapModeOnMatch); + } + + @Override + public Map> select(List>> partitions) { + return partitions.stream() + .map(p -> Pair.of(pattern.matcher(p.getKey()).matches() ? bootstrapModeOnMatch : defaultMode, p.getKey())) + .collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toList()))); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java new file mode 100644 index 0000000000000..18e5b38a18f4e --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap.selector; + +import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.config.HoodieWriteConfig; + +public class FullRecordBootstrapModeSelector extends UniformBootstrapModeSelector { + + public FullRecordBootstrapModeSelector(HoodieWriteConfig bootstrapConfig) { + super(bootstrapConfig, BootstrapMode.FULL_RECORD); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java new file mode 100644 index 0000000000000..5de95e4420fd8 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap.selector; + +import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.config.HoodieWriteConfig; + +public class MetadataOnlyBootstrapModeSelector extends UniformBootstrapModeSelector { + + public MetadataOnlyBootstrapModeSelector(HoodieWriteConfig bootstrapConfig) { + super(bootstrapConfig, BootstrapMode.METADATA_ONLY); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java new file mode 100644 index 0000000000000..e2784c755635e --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap.selector; + +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * A bootstrap selector which employs same bootstrap mode for all partitions. + */ +public abstract class UniformBootstrapModeSelector extends BootstrapModeSelector { + + private final BootstrapMode bootstrapMode; + + public UniformBootstrapModeSelector(HoodieWriteConfig bootstrapConfig, BootstrapMode bootstrapMode) { + super(bootstrapConfig); + this.bootstrapMode = bootstrapMode; + } + + @Override + public Map> select(List>> partitions) { + return partitions.stream().map(p -> Pair.of(bootstrapMode, p)) + .collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(x -> x.getValue().getKey(), + Collectors.toList()))); + } +} \ No newline at end of file diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java new file mode 100644 index 0000000000000..6f46e8e7667dd --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap.translator; + +import java.io.Serializable; +import org.apache.hudi.common.config.TypedProperties; + +public abstract class BootstrapPartitionPathTranslator implements Serializable { + + private final TypedProperties properties; + + public BootstrapPartitionPathTranslator(TypedProperties properties) { + this.properties = properties; + } + + /** + * Given a bootstrap partition path, translated partition path. + * + * @param bootStrapPartitionPath bootstrap Partition Path + * @return Translated Path + */ + public abstract String getBootstrapTranslatedPath(String bootStrapPartitionPath); +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java new file mode 100644 index 0000000000000..a433eb5682da8 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap.translator; + +import org.apache.hudi.common.config.TypedProperties; + +/** + * Return same path as bootstrap partition path. + */ +public class IdentityBootstrapPartitionPathTranslator extends BootstrapPartitionPathTranslator { + + public IdentityBootstrapPartitionPathTranslator(TypedProperties properties) { + super(properties); + } + + @Override + public String getBootstrapTranslatedPath(String bootStrapPartitionPath) { + return bootStrapPartitionPath; + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/MergingIterator.java b/hudi-client/src/main/java/org/apache/hudi/client/utils/MergingIterator.java new file mode 100644 index 0000000000000..47dde723e00dd --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/utils/MergingIterator.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.utils; + +import java.util.Iterator; +import java.util.function.Function; +import org.apache.avro.generic.GenericRecord; + +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; + +public class MergingIterator implements Iterator { + + private final Iterator leftIterator; + private final Iterator rightIterator; + private final Function, T> mergeFunction; + + public MergingIterator(Iterator leftIterator, Iterator rightIterator, Function, T> mergeFunction) { + this.leftIterator = leftIterator; + this.rightIterator = rightIterator; + this.mergeFunction = mergeFunction; + } + + @Override + public boolean hasNext() { + boolean leftHasNext = leftIterator.hasNext(); + boolean rightHasNext = rightIterator.hasNext(); + ValidationUtils.checkArgument(leftHasNext == rightHasNext); + return leftHasNext; + } + + @Override + public T next() { + return mergeFunction.apply(Pair.of(leftIterator.next(), rightIterator.next())); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java b/hudi-client/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java new file mode 100644 index 0000000000000..ebfaed0d3b196 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.config; + +import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector; +import org.apache.hudi.client.bootstrap.translator.IdentityBootstrapPartitionPathTranslator; +import org.apache.hudi.common.config.DefaultHoodieConfig; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +/** + * Bootstrap specific configs. + */ +public class HoodieBootstrapConfig extends DefaultHoodieConfig { + + public static final String BOOTSTRAP_BASE_PATH_PROP = "hoodie.bootstrap.base.path"; + public static final String BOOTSTRAP_MODE_SELECTOR = "hoodie.bootstrap.mode.selector"; + public static final String FULL_BOOTSTRAP_INPUT_PROVIDER = "hoodie.bootstrap.full.input.provider"; + public static final String BOOTSTRAP_KEYGEN_CLASS = "hoodie.bootstrap.keygen.class"; + public static final String BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS = + "hoodie.bootstrap.partitionpath.translator.class"; + public static final String DEFAULT_BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS = + IdentityBootstrapPartitionPathTranslator.class.getName(); + + public static final String BOOTSTRAP_PARALLELISM = "hoodie.bootstrap.parallelism"; + public static final String DEFAULT_BOOTSTRAP_PARALLELISM = "1500"; + + // Used By BootstrapRegexModeSelector class. When a partition path matches the regex, the corresponding + // mode will be used. Otherwise, the alternative mode will be used. + public static final String BOOTSTRAP_MODE_SELECTOR_REGEX = "hoodie.bootstrap.mode.selector.regex"; + public static final String BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = "hoodie.bootstrap.mode.selector.regex.mode"; + public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX = ".*"; + public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = BootstrapMode.METADATA_ONLY.name(); + + public HoodieBootstrapConfig(Properties props) { + super(props); + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + + private final Properties props = new Properties(); + + public Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + this.props.load(reader); + return this; + } + } + + public Builder withBootstrapBasePath(String basePath) { + props.setProperty(BOOTSTRAP_BASE_PATH_PROP, basePath); + return this; + } + + public Builder withBootstrapModeSelector(String partitionSelectorClass) { + props.setProperty(BOOTSTRAP_MODE_SELECTOR, partitionSelectorClass); + return this; + } + + public Builder withFullBootstrapInputProvider(String partitionSelectorClass) { + props.setProperty(FULL_BOOTSTRAP_INPUT_PROVIDER, partitionSelectorClass); + return this; + } + + public Builder withBootstrapKeyGenClass(String keyGenClass) { + props.setProperty(BOOTSTRAP_KEYGEN_CLASS, keyGenClass); + return this; + } + + public Builder withBootstrapPartitionPathTranslatorClass(String partitionPathTranslatorClass) { + props.setProperty(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, partitionPathTranslatorClass); + return this; + } + + public Builder withBootstrapParallelism(int parallelism) { + props.setProperty(BOOTSTRAP_PARALLELISM, String.valueOf(parallelism)); + return this; + } + + public Builder withBootstrapModeSelectorRegex(String regex) { + props.setProperty(BOOTSTRAP_MODE_SELECTOR_REGEX, regex); + return this; + } + + public Builder withBootstrapModeForRegexMatch(BootstrapMode modeForRegexMatch) { + props.setProperty(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, modeForRegexMatch.name()); + return this; + } + + public Builder fromProperties(Properties props) { + this.props.putAll(props); + return this; + } + + public HoodieBootstrapConfig build() { + HoodieBootstrapConfig config = new HoodieBootstrapConfig(props); + setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_PARALLELISM), BOOTSTRAP_PARALLELISM, + DEFAULT_BOOTSTRAP_PARALLELISM); + setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS), + BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, DEFAULT_BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS); + setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR), BOOTSTRAP_MODE_SELECTOR, + MetadataOnlyBootstrapModeSelector.class.getCanonicalName()); + setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR_REGEX), BOOTSTRAP_MODE_SELECTOR_REGEX, + DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX); + setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE), + BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE); + BootstrapMode.valueOf(props.getProperty(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE)); + return config; + } + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 61e89e3daf904..affe553522198 100644 --- a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.HoodieWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.bootstrap.BootstrapMode; import org.apache.hudi.common.config.DefaultHoodieConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.HoodieCleaningPolicy; @@ -129,6 +130,9 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { "_.hoodie.allow.multi.write.on.same.instant"; public static final String DEFAULT_ALLOW_MULTI_WRITE_ON_SAME_INSTANT = "false"; + public static final String EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION = AVRO_SCHEMA + ".externalTransformation"; + public static final String DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION = "false"; + private ConsistencyGuardConfig consistencyGuardConfig; // Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled @@ -136,7 +140,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private final FileSystemViewStorageConfig clientSpecifiedViewStorageConfig; private FileSystemViewStorageConfig viewStorageConfig; - private HoodieWriteConfig(Properties props) { + protected HoodieWriteConfig(Properties props) { super(props); Properties newProps = new Properties(); newProps.putAll(props); @@ -180,6 +184,10 @@ public Boolean shouldAssumeDatePartitioning() { return Boolean.parseBoolean(props.getProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP)); } + public boolean shouldUseExternalSchemaTransformation() { + return Boolean.parseBoolean(props.getProperty(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION)); + } + public Integer getTimelineLayoutVersion() { return Integer.parseInt(props.getProperty(TIMELINE_LAYOUT_VERSION)); } @@ -675,13 +683,46 @@ public String getCallbackClass() { return props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP); } + public String getBootstrapSourceBasePath() { + return props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP); + } + + public String getBootstrapModeSelectorClass() { + return props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR); + } + + public String getFullBootstrapInputProvider() { + return props.getProperty(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER); + } + + public String getBootstrapKeyGeneratorClass() { + return props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS); + } + + public String getBootstrapModeSelectorRegex() { + return props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX); + } + + public BootstrapMode getBootstrapModeForRegexMatch() { + return BootstrapMode.valueOf(props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX_MODE)); + } + + public String getBootstrapPartitionPathTranslatorClass() { + return props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS); + } + + public int getBootstrapParallelism() { + return Integer.parseInt(props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM)); + } + public static class Builder { - private final Properties props = new Properties(); + protected final Properties props = new Properties(); private boolean isIndexConfigSet = false; private boolean isStorageConfigSet = false; private boolean isCompactionConfigSet = false; private boolean isMetricsConfigSet = false; + private boolean isBootstrapConfigSet = false; private boolean isMemoryConfigSet = false; private boolean isViewConfigSet = false; private boolean isConsistencyGuardSet = false; @@ -805,6 +846,12 @@ public Builder withMemoryConfig(HoodieMemoryConfig memoryConfig) { return this; } + public Builder withBootstrapConfig(HoodieBootstrapConfig bootstrapConfig) { + props.putAll(bootstrapConfig.getProps()); + isBootstrapConfigSet = true; + return this; + } + public Builder withAutoCommit(boolean autoCommit) { props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit)); return this; @@ -863,7 +910,17 @@ public Builder withAllowMultiWriteOnSameInstant(boolean allow) { return this; } - public HoodieWriteConfig build() { + public Builder withExternalSchemaTrasformation(boolean enabled) { + props.setProperty(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, String.valueOf(enabled)); + return this; + } + + public Builder withProperties(Properties properties) { + this.props.putAll(properties); + return this; + } + + protected void setDefaults() { // Check for mandatory properties setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM, DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM), BULKINSERT_PARALLELISM, @@ -916,6 +973,8 @@ public HoodieWriteConfig build() { setDefaultOnCondition(props, !isCompactionConfigSet, HoodieCompactionConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isMetricsConfigSet, HoodieMetricsConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !isBootstrapConfigSet, + HoodieBootstrapConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isMemoryConfigSet, HoodieMemoryConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isViewConfigSet, FileSystemViewStorageConfig.newBuilder().fromProperties(props).build()); @@ -924,15 +983,24 @@ public HoodieWriteConfig build() { setDefaultOnCondition(props, !isCallbackConfigSet, HoodieWriteCommitCallbackConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !props.containsKey(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION), + EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION); setDefaultOnCondition(props, !props.containsKey(TIMELINE_LAYOUT_VERSION), TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION)); + } + + private void validate() { String layoutVersion = props.getProperty(TIMELINE_LAYOUT_VERSION); // Ensure Layout Version is good new TimelineLayoutVersion(Integer.parseInt(layoutVersion)); + Objects.requireNonNull(props.getProperty(BASE_PATH_PROP)); + } + public HoodieWriteConfig build() { + setDefaults(); + validate(); // Build WriteConfig at the end HoodieWriteConfig config = new HoodieWriteConfig(props); - Objects.requireNonNull(config.getBasePath()); return config; } } 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 e81c4add75415..7a8e5abf367e1 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 @@ -165,7 +165,7 @@ private void init(HoodieRecord record) { private Option getIndexedRecord(HoodieRecord hoodieRecord) { Option recordMetadata = hoodieRecord.getData().getMetadata(); try { - Option avroRecord = hoodieRecord.getData().getInsertValue(originalSchema); + Option avroRecord = hoodieRecord.getData().getInsertValue(writerSchema); if (avroRecord.isPresent()) { // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get())); @@ -212,7 +212,7 @@ public void doAppend() { private void doAppend(Map header) { try { header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime); - header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writerSchema.toString()); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writerSchemaWithMetafields.toString()); if (recordList.size() > 0) { writer = writer.appendBlock(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header)); recordList.clear(); diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java new file mode 100644 index 0000000000000..5deeae1d9f228 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +/** + * This class is essentially same as Create Handle but overrides two things + * 1) Schema : Metadata bootstrap writes only metadata fields as part of write. So, setup the writer schema accordingly. + * 2) canWrite is overridden to always return true so that skeleton file and bootstrap file is aligned and we don't end up + * writing more than 1 skeleton file for the same bootstrap file. + * @param HoodieRecordPayload + */ +public class HoodieBootstrapHandle extends HoodieCreateHandle { + + public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) { + super(config, commitTime, hoodieTable, partitionPath, fileId, + Pair.of(HoodieAvroUtils.RECORD_KEY_SCHEMA, + HoodieAvroUtils.addMetadataFields(HoodieAvroUtils.RECORD_KEY_SCHEMA)), sparkTaskContextSupplier); + } + + @Override + public boolean canWrite(HoodieRecord record) { + return true; + } +} \ No newline at end of file diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index dfa63b0797a6f..705e98d944e36 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -18,6 +18,7 @@ package org.apache.hudi.io; +import org.apache.avro.Schema; import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.fs.FSUtils; @@ -28,6 +29,7 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieInsertException; import org.apache.hudi.io.storage.HoodieFileWriter; @@ -56,8 +58,16 @@ public class HoodieCreateHandle extends HoodieWri private boolean useWriterSchema = false; public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, - String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) { - super(config, instantTime, partitionPath, fileId, hoodieTable, sparkTaskContextSupplier); + String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) { + this(config, instantTime, hoodieTable, partitionPath, fileId, getWriterSchemaIncludingAndExcludingMetadataPair(config), + sparkTaskContextSupplier); + } + + public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, + String partitionPath, String fileId, Pair writerSchemaIncludingAndExcludingMetadataPair, + SparkTaskContextSupplier sparkTaskContextSupplier) { + super(config, instantTime, partitionPath, fileId, hoodieTable, writerSchemaIncludingAndExcludingMetadataPair, + sparkTaskContextSupplier); writeStatus.setFileId(fileId); writeStatus.setPartitionPath(partitionPath); @@ -68,8 +78,7 @@ public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTa new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath)); partitionMetadata.trySave(getPartitionId()); createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension())); - this.fileWriter = - HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writerSchema, this.sparkTaskContextSupplier); + this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writerSchemaWithMetafields, this.sparkTaskContextSupplier); } catch (IOException e) { throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e); } @@ -132,9 +141,9 @@ public void write() { while (recordIterator.hasNext()) { HoodieRecord record = recordIterator.next(); if (useWriterSchema) { - write(record, record.getData().getInsertValue(writerSchema)); + write(record, record.getData().getInsertValue(writerSchemaWithMetafields)); } else { - write(record, record.getData().getInsertValue(originalSchema)); + write(record, record.getData().getInsertValue(writerSchema)); } } } catch (IOException io) { 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 e87cf3c01474a..f0ea284eac299 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 @@ -67,6 +67,7 @@ public class HoodieMergeHandle extends HoodieWrit private long updatedRecordsWritten = 0; private long insertRecordsWritten = 0; private boolean useWriterSchema; + private HoodieBaseFile baseFileToMerge; public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, Iterator> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) { @@ -88,6 +89,10 @@ public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTab } @Override + public Schema getWriterSchemaWithMetafields() { + return writerSchemaWithMetafields; + } + public Schema getWriterSchema() { return writerSchema; } @@ -95,12 +100,13 @@ public Schema getWriterSchema() { /** * Extract old file path, initialize StorageWriter and WriteStatus. */ - private void init(String fileId, String partitionPath, HoodieBaseFile dataFileToBeMerged) { + private void init(String fileId, String partitionPath, HoodieBaseFile baseFileToMerge) { LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" + fileId); + this.baseFileToMerge = baseFileToMerge; this.writtenRecordKeys = new HashSet<>(); writeStatus.setStat(new HoodieWriteStat()); try { - String latestValidFilePath = dataFileToBeMerged.getFileName(); + String latestValidFilePath = baseFileToMerge.getFileName(); writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath)); HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, instantTime, @@ -126,8 +132,7 @@ private void init(String fileId, String partitionPath, HoodieBaseFile dataFileTo createMarkerFile(partitionPath, newFileName); // Create the writer for writing the new version file - fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config, writerSchema, sparkTaskContextSupplier); - + fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config, writerSchemaWithMetafields, sparkTaskContextSupplier); } catch (IOException io) { LOG.error("Error in update task at commit " + instantTime, io); writeStatus.setGlobalError(io); @@ -145,7 +150,7 @@ private void init(String fileId, Iterator> newRecordsItr) { long memoryForMerge = SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps()); LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge); this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(), - new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(originalSchema)); + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(writerSchema)); } catch (IOException io) { throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io); } @@ -216,7 +221,7 @@ public void write(GenericRecord oldRecord) { HoodieRecord hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key)); try { Option combinedAvroRecord = - hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchema : originalSchema); + hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchemaWithMetafields : writerSchema); if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) { /* * ONLY WHEN 1) we have an update for this key AND 2) We are able to successfully write the the combined new @@ -241,7 +246,7 @@ public void write(GenericRecord oldRecord) { fileWriter.writeAvro(key, oldRecord); } catch (ClassCastException e) { LOG.error("Schema mismatch when rewriting old record " + oldRecord + " from file " + getOldFilePath() - + " to file " + newFilePath + " with writerSchema " + writerSchema.toString(true)); + + " to file " + newFilePath + " with writerSchema " + writerSchemaWithMetafields.toString(true)); throw new HoodieUpsertException(errMsg, e); } catch (IOException e) { LOG.error("Failed to merge old record into new file for key " + key + " from old file " + getOldFilePath() @@ -262,9 +267,9 @@ public WriteStatus close() { HoodieRecord hoodieRecord = newRecordsItr.next(); if (!writtenRecordKeys.contains(hoodieRecord.getRecordKey())) { if (useWriterSchema) { - writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema)); + writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchemaWithMetafields)); } else { - writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(originalSchema)); + writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema)); } insertRecordsWritten++; } @@ -312,4 +317,8 @@ public WriteStatus getWriteStatus() { public IOType getIOType() { return IOType.MERGE; } + + public HoodieBaseFile baseFileForMerge() { + return baseFileToMerge; + } } diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 7fd3b42aec745..d148b1b8ac7f7 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieFileWriter; @@ -51,8 +52,8 @@ public abstract class HoodieWriteHandle extends H private static final Logger LOG = LogManager.getLogger(HoodieWriteHandle.class); - protected final Schema originalSchema; protected final Schema writerSchema; + protected final Schema writerSchemaWithMetafields; protected HoodieTimer timer; protected final WriteStatus writeStatus; protected final String partitionPath; @@ -62,11 +63,18 @@ public abstract class HoodieWriteHandle extends H public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, String fileId, HoodieTable hoodieTable, SparkTaskContextSupplier sparkTaskContextSupplier) { + this(config, instantTime, partitionPath, fileId, hoodieTable, + getWriterSchemaIncludingAndExcludingMetadataPair(config), sparkTaskContextSupplier); + } + + protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, String fileId, + HoodieTable hoodieTable, Pair writerSchemaIncludingAndExcludingMetadataPair, + SparkTaskContextSupplier sparkTaskContextSupplier) { super(config, instantTime, hoodieTable); this.partitionPath = partitionPath; this.fileId = fileId; - this.originalSchema = new Schema.Parser().parse(config.getSchema()); - this.writerSchema = HoodieAvroUtils.createHoodieWriteSchema(originalSchema); + this.writerSchema = writerSchemaIncludingAndExcludingMetadataPair.getKey(); + this.writerSchemaWithMetafields = writerSchemaIncludingAndExcludingMetadataPair.getValue(); this.timer = new HoodieTimer().startTimer(); this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(), !hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction()); @@ -74,6 +82,19 @@ public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String pa this.writeToken = makeWriteToken(); } + /** + * Returns writer schema pairs containing + * (a) Writer Schema from client + * (b) (a) with hoodie metadata fields. + * @param config Write Config + * @return + */ + protected static Pair getWriterSchemaIncludingAndExcludingMetadataPair(HoodieWriteConfig config) { + Schema originalSchema = new Schema.Parser().parse(config.getSchema()); + Schema hoodieSchema = HoodieAvroUtils.addMetadataFields(originalSchema); + return Pair.of(originalSchema, hoodieSchema); + } + /** * Generate a write token based on the currently running spark task and its place in the spark dag. */ @@ -103,8 +124,8 @@ protected void createMarkerFile(String partitionPath, String dataFileName) { markerFiles.create(partitionPath, dataFileName, getIOType()); } - public Schema getWriterSchema() { - return writerSchema; + public Schema getWriterSchemaWithMetafields() { + return writerSchemaWithMetafields; } /** @@ -142,7 +163,7 @@ public void write(HoodieRecord record, Option avroRecord, Option< * Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields. */ protected GenericRecord rewriteRecord(GenericRecord record) { - return HoodieAvroUtils.rewriteRecord(record, writerSchema); + return HoodieAvroUtils.rewriteRecord(record, writerSchemaWithMetafields); } public abstract WriteStatus close(); diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java new file mode 100644 index 0000000000000..7c3edf747b557 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.keygen; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.exception.HoodieKeyException; + +import org.apache.avro.generic.GenericRecord; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * Base class for all the built-in key generators. Contains methods structured for + * code reuse amongst them. + */ +public abstract class BuiltinKeyGenerator extends KeyGenerator { + + protected BuiltinKeyGenerator(TypedProperties config) { + super(config); + } + + /** + * Generate a record Key out of provided generic record. + */ + public abstract String getRecordKey(GenericRecord record); + + /** + * Generate a partition path out of provided generic record. + */ + public abstract String getPartitionPath(GenericRecord record); + + /** + * Generate a Hoodie Key out of provided generic record. + */ + public final HoodieKey getKey(GenericRecord record) { + if (getRecordKeyFields() == null || getPartitionPathFields() == null) { + throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg"); + } + return new HoodieKey(getRecordKey(record), getPartitionPath(record)); + } + + /** + * Return fields that constitute record key. Used by Metadata bootstrap. + * Have a base implementation inorder to prevent forcing custom KeyGenerator implementation + * to implement this method + * @return list of record key fields + */ + public List getRecordKeyFields() { + throw new IllegalStateException("This method is expected to be overridden by subclasses"); + } + + /** + * Return fields that constiture partition path. Used by Metadata bootstrap. + * Have a base implementation inorder to prevent forcing custom KeyGenerator implementation + * to implement this method + * @return list of partition path fields + */ + public List getPartitionPathFields() { + throw new IllegalStateException("This method is expected to be overridden by subclasses"); + } + + @Override + public final List getRecordKeyFieldNames() { + // For nested columns, pick top level column name + return getRecordKeyFields().stream().map(k -> { + int idx = k.indexOf('.'); + return idx > 0 ? k.substring(0, idx) : k; + }).collect(Collectors.toList()); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java b/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java new file mode 100644 index 0000000000000..c4ac29b2b1fe3 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.keygen; + +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.List; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieKeyException; + +public class KeyGenUtils { + + protected static final String NULL_RECORDKEY_PLACEHOLDER = "__null__"; + protected static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__"; + + protected static final String DEFAULT_PARTITION_PATH = "default"; + protected static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/"; + + public static String getRecordKey(GenericRecord record, List recordKeyFields) { + boolean keyIsNullEmpty = true; + StringBuilder recordKey = new StringBuilder(); + for (String recordKeyField : recordKeyFields) { + String recordKeyValue = HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyField, true); + if (recordKeyValue == null) { + recordKey.append(recordKeyField + ":" + NULL_RECORDKEY_PLACEHOLDER + ","); + } else if (recordKeyValue.isEmpty()) { + recordKey.append(recordKeyField + ":" + EMPTY_RECORDKEY_PLACEHOLDER + ","); + } else { + recordKey.append(recordKeyField + ":" + recordKeyValue + ","); + keyIsNullEmpty = false; + } + } + recordKey.deleteCharAt(recordKey.length() - 1); + if (keyIsNullEmpty) { + throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: " + + recordKeyFields.toString() + " cannot be entirely null or empty."); + } + return recordKey.toString(); + } + + public static String getRecordPartitionPath(GenericRecord record, List partitionPathFields, + boolean hiveStylePartitioning, boolean encodePartitionPath) { + StringBuilder partitionPath = new StringBuilder(); + for (String partitionPathField : partitionPathFields) { + String fieldVal = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true); + if (fieldVal == null || fieldVal.isEmpty()) { + partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + DEFAULT_PARTITION_PATH + : DEFAULT_PARTITION_PATH); + } else { + if (encodePartitionPath) { + try { + fieldVal = URLEncoder.encode(fieldVal, StandardCharsets.UTF_8.toString()); + } catch (UnsupportedEncodingException uoe) { + throw new HoodieException(uoe.getMessage(), uoe); + } + } + partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + fieldVal : fieldVal); + } + partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR); + } + partitionPath.deleteCharAt(partitionPath.length() - 1); + return partitionPath.toString(); + } + + public static String getRecordKey(GenericRecord record, String recordKeyField) { + String recordKey = HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyField, true); + if (recordKey == null || recordKey.isEmpty()) { + throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty."); + } + return recordKey; + } + + public static String getPartitionPath(GenericRecord record, String partitionPathField, + boolean hiveStylePartitioning, boolean encodePartitionPath) { + String partitionPath = HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathField, true); + if (partitionPath == null || partitionPath.isEmpty()) { + partitionPath = DEFAULT_PARTITION_PATH; + } + if (encodePartitionPath) { + try { + partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString()); + } catch (UnsupportedEncodingException uoe) { + throw new HoodieException(uoe.getMessage(), uoe); + } + } + if (hiveStylePartitioning) { + partitionPath = partitionPathField + "=" + partitionPath; + } + return partitionPath; + } +} \ No newline at end of file diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java similarity index 76% rename from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java rename to hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java index b4d609dcf3dd8..1a798af7485e4 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java +++ b/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGenerator.java @@ -24,6 +24,7 @@ import org.apache.avro.generic.GenericRecord; import java.io.Serializable; +import java.util.List; /** * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record. @@ -40,4 +41,14 @@ protected KeyGenerator(TypedProperties config) { * Generate a Hoodie Key out of provided generic record. */ public abstract HoodieKey getKey(GenericRecord record); -} \ No newline at end of file + + /** + * Used during bootstrap, to project out only the record key fields from bootstrap source dataset. + * + * @return list of field names, when concatenated make up the record key. + */ + public List getRecordKeyFieldNames() { + throw new UnsupportedOperationException("Bootstrap not supported for key generator. " + + "Please override this method in your custom key generator."); + } +} 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 849673eaaba70..21f8bef07912f 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 @@ -18,9 +18,6 @@ package org.apache.hudi.table; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -33,30 +30,32 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.exception.HoodieUpsertException; -import org.apache.hudi.execution.SparkBoundedInMemoryExecutor; import org.apache.hudi.io.HoodieCreateHandle; import org.apache.hudi.io.HoodieMergeHandle; -import org.apache.hudi.io.storage.HoodieFileReader; -import org.apache.hudi.io.storage.HoodieFileReaderFactory; -import org.apache.hudi.table.action.clean.CleanActionExecutor; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.bootstrap.BootstrapCommitActionExecutor; +import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; +import org.apache.hudi.table.action.clean.CleanActionExecutor; import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor; import org.apache.hudi.table.action.commit.BulkInsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.commit.DeleteCommitActionExecutor; import org.apache.hudi.table.action.commit.InsertCommitActionExecutor; import org.apache.hudi.table.action.commit.InsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.MergeHelper; import org.apache.hudi.table.action.commit.UpsertCommitActionExecutor; import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor; import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -97,7 +96,7 @@ public HoodieWriteMetadata insert(JavaSparkContext jsc, String instantTime, Java @Override public HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, JavaRDD> records, Option userDefinedBulkInsertPartitioner) { - return new BulkInsertCommitActionExecutor<>(jsc, config, + return new BulkInsertCommitActionExecutor(jsc, config, this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); } @@ -121,7 +120,7 @@ public HoodieWriteMetadata insertPrepped(JavaSparkContext jsc, String instantTim @Override public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime, JavaRDD> preppedRecords, Option userDefinedBulkInsertPartitioner) { - return new BulkInsertPreppedCommitActionExecutor<>(jsc, config, + return new BulkInsertPreppedCommitActionExecutor(jsc, config, this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); } @@ -135,6 +134,16 @@ public HoodieWriteMetadata compact(JavaSparkContext jsc, String compactionInstan throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); } + @Override + public HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option> extraMetadata) { + return new BootstrapCommitActionExecutor(jsc, config, this, extraMetadata).execute(); + } + + @Override + public void rollbackBootstrap(JavaSparkContext jsc, String instantTime) { + new CopyOnWriteRestoreActionExecutor(jsc, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); + } + public Iterator> handleUpdate(String instantTime, String partitionPath, String fileId, Map> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException { // these are updates @@ -148,25 +157,10 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups throw new HoodieUpsertException( "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); } else { - BoundedInMemoryExecutor wrapper = null; - HoodieFileReader storageReader = - HoodieFileReaderFactory.getFileReader(getHadoopConf(), upsertHandle.getOldFilePath()); - - try { - wrapper = - new SparkBoundedInMemoryExecutor(config, storageReader.getRecordIterator(upsertHandle.getWriterSchema()), - new UpdateHandler(upsertHandle), x -> x); - wrapper.execute(); - } catch (Exception e) { - throw new HoodieException(e); - } finally { - upsertHandle.close(); - if (null != wrapper) { - wrapper.shutdownNow(); - } - } + MergeHelper.runMerge(this, upsertHandle); } + // TODO(vc): This needs to be revisited if (upsertHandle.getWriteStatus().getPartitionPath() == null) { LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " 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 785efa57976a0..a236cdb9411de 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 @@ -28,10 +28,14 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; + +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.bootstrap.BootstrapDeltaCommitActionExecutor; +import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.compact.RunCompactionActionExecutor; import org.apache.hudi.table.action.deltacommit.BulkInsertDeltaCommitActionExecutor; import org.apache.hudi.table.action.deltacommit.BulkInsertPreppedDeltaCommitActionExecutor; @@ -84,7 +88,7 @@ public HoodieWriteMetadata insert(JavaSparkContext jsc, String instantTime, Java @Override public HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, JavaRDD> records, Option userDefinedBulkInsertPartitioner) { - return new BulkInsertDeltaCommitActionExecutor<>(jsc, config, + return new BulkInsertDeltaCommitActionExecutor(jsc, config, this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); } @@ -108,7 +112,7 @@ public HoodieWriteMetadata insertPrepped(JavaSparkContext jsc, String instantTim @Override public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime, JavaRDD> preppedRecords, Option userDefinedBulkInsertPartitioner) { - return new BulkInsertPreppedDeltaCommitActionExecutor<>(jsc, config, + return new BulkInsertPreppedDeltaCommitActionExecutor(jsc, config, this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); } @@ -125,6 +129,16 @@ public HoodieWriteMetadata compact(JavaSparkContext jsc, String compactionInstan return compactionExecutor.execute(); } + @Override + public HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option> extraMetadata) { + return new BootstrapDeltaCommitActionExecutor(jsc, config, this, extraMetadata).execute(); + } + + @Override + public void rollbackBootstrap(JavaSparkContext jsc, String instantTime) { + new MergeOnReadRestoreActionExecutor(jsc, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); + } + @Override public HoodieRollbackMetadata rollback(JavaSparkContext jsc, String rollbackInstantTime, 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 d8b0c6e9018e1..565e046ea1da2 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 @@ -60,6 +60,7 @@ import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -331,6 +332,20 @@ public abstract Option scheduleCompaction(JavaSparkContext public abstract HoodieWriteMetadata compact(JavaSparkContext jsc, String compactionInstantTime); + /** + * Perform metadata/full bootstrap of a Hudi table. + * @param jsc JavaSparkContext + * @param extraMetadata Additional Metadata for storing in commit file. + * @return HoodieBootstrapWriteMetadata + */ + public abstract HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option> extraMetadata); + + /** + * Perform rollback of bootstrap of a Hudi table. + * @param jsc JavaSparkContext + */ + public abstract void rollbackBootstrap(JavaSparkContext jsc, String instantTime); + /** * Executes a new clean action. * diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java new file mode 100644 index 0000000000000..e4224fdd8470d --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java @@ -0,0 +1,356 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.bootstrap; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.client.bootstrap.BootstrapRecordPayload; +import org.apache.hudi.client.bootstrap.BootstrapSchemaProvider; +import org.apache.hudi.client.bootstrap.BootstrapWriteStatus; +import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; +import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector; +import org.apache.hudi.client.bootstrap.translator.BootstrapPartitionPathTranslator; +import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.bootstrap.index.BootstrapIndex; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.BootstrapFileMapping; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ParquetReaderIterator; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.execution.SparkBoundedInMemoryExecutor; +import org.apache.hudi.io.HoodieBootstrapHandle; +import org.apache.hudi.keygen.KeyGenerator; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.commit.BaseCommitActionExecutor; +import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.table.action.commit.CommitActionExecutor; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.avro.AvroReadSupport; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class BootstrapCommitActionExecutor> + extends BaseCommitActionExecutor { + + private static final Logger LOG = LogManager.getLogger(BootstrapCommitActionExecutor.class); + protected String bootstrapSchema = null; + private transient FileSystem bootstrapSourceFileSystem; + + public BootstrapCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, + Option> extraMetadata) { + super(jsc, new HoodieWriteConfig.Builder().withProps(config.getProps()) + .withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class) + .withBulkInsertParallelism(config.getBootstrapParallelism()) + .build(), table, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, WriteOperationType.BOOTSTRAP, + extraMetadata); + bootstrapSourceFileSystem = FSUtils.getFs(config.getBootstrapSourceBasePath(), hadoopConf); + } + + private void validate() { + ValidationUtils.checkArgument(config.getBootstrapSourceBasePath() != null, + "Ensure Bootstrap Source Path is set"); + ValidationUtils.checkArgument(config.getBootstrapModeSelectorClass() != null, + "Ensure Bootstrap Partition Selector is set"); + ValidationUtils.checkArgument(config.getBootstrapKeyGeneratorClass() != null, + "Ensure bootstrap key generator class is set"); + } + + @Override + public HoodieBootstrapWriteMetadata execute() { + validate(); + try { + HoodieTableMetaClient metaClient = table.getMetaClient(); + Option completetedInstant = + metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); + ValidationUtils.checkArgument(!completetedInstant.isPresent(), + "Active Timeline is expected to be empty for bootstrap to be performed. " + + "If you want to re-bootstrap, please rollback bootstrap first !!"); + Map>>> partitionSelections = listAndProcessSourcePartitions(); + + // First run metadata bootstrap which will auto commit + Option metadataResult = metadataBootstrap(partitionSelections.get(BootstrapMode.METADATA_ONLY)); + // if there are full bootstrap to be performed, perform that too + Option fullBootstrapResult = fullBootstrap(partitionSelections.get(BootstrapMode.FULL_RECORD)); + return new HoodieBootstrapWriteMetadata(metadataResult, fullBootstrapResult); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + protected String getSchemaToStoreInCommit() { + return bootstrapSchema; + } + + /** + * Perform Metadata Bootstrap. + * @param partitionFilesList List of partitions and files within that partitions + */ + protected Option metadataBootstrap(List>> partitionFilesList) { + if (null == partitionFilesList || partitionFilesList.isEmpty()) { + return Option.empty(); + } + + HoodieTableMetaClient metaClient = table.getMetaClient(); + metaClient.getActiveTimeline().createNewInstant( + new HoodieInstant(State.REQUESTED, metaClient.getCommitActionType(), + HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)); + + table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, + metaClient.getCommitActionType(), HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS), Option.empty()); + + JavaRDD bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList); + + HoodieWriteMetadata result = new HoodieWriteMetadata(); + updateIndexAndCommitIfNeeded(bootstrapWriteStatuses.map(w -> w), result); + return Option.of(result); + } + + @Override + protected void commit(Option> extraMetadata, HoodieWriteMetadata result) { + // Perform bootstrap index write and then commit. Make sure both record-key and bootstrap-index + // is all done in a single job DAG. + Map>> bootstrapSourceAndStats = + result.getWriteStatuses().collect().stream() + .map(w -> { + BootstrapWriteStatus ws = (BootstrapWriteStatus) w; + return Pair.of(ws.getBootstrapSourceFileMapping(), ws.getStat()); + }).collect(Collectors.groupingBy(w -> w.getKey().getPartitionPath())); + HoodieTableMetaClient metaClient = table.getMetaClient(); + try (BootstrapIndex.IndexWriter indexWriter = BootstrapIndex.getBootstrapIndex(metaClient) + .createWriter(metaClient.getTableConfig().getBootstrapBasePath().get())) { + LOG.info("Starting to write bootstrap index for source " + config.getBootstrapSourceBasePath() + " in table " + + config.getBasePath()); + indexWriter.begin(); + bootstrapSourceAndStats.forEach((key, value) -> indexWriter.appendNextPartition(key, + value.stream().map(Pair::getKey).collect(Collectors.toList()))); + indexWriter.finish(); + LOG.info("Finished writing bootstrap index for source " + config.getBootstrapSourceBasePath() + " in table " + + config.getBasePath()); + } + + super.commit(extraMetadata, result, bootstrapSourceAndStats.values().stream() + .flatMap(f -> f.stream().map(Pair::getValue)).collect(Collectors.toList())); + LOG.info("Committing metadata bootstrap !!"); + } + + /** + * Perform Metadata Bootstrap. + * @param partitionFilesList List of partitions and files within that partitions + */ + protected Option fullBootstrap(List>> partitionFilesList) { + if (null == partitionFilesList || partitionFilesList.isEmpty()) { + return Option.empty(); + } + TypedProperties properties = new TypedProperties(); + properties.putAll(config.getProps()); + FullRecordBootstrapDataProvider inputProvider = + (FullRecordBootstrapDataProvider) ReflectionUtils.loadClass(config.getFullBootstrapInputProvider(), + properties, jsc); + JavaRDD inputRecordsRDD = + inputProvider.generateInputRecordRDD("bootstrap_source", config.getBootstrapSourceBasePath(), + partitionFilesList); + // Start Full Bootstrap + final HoodieInstant requested = new HoodieInstant(State.REQUESTED, table.getMetaClient().getCommitActionType(), + HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); + table.getActiveTimeline().createNewInstant(requested); + + // Setup correct schema and run bulk insert. + return Option.of(getBulkInsertActionExecutor(inputRecordsRDD).execute()); + } + + protected CommitActionExecutor getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { + return new BulkInsertCommitActionExecutor(jsc, new HoodieWriteConfig.Builder().withProps(config.getProps()) + .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, + inputRecordsRDD, extraMetadata); + } + + private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, String partitionPath, + HoodieFileStatus srcFileStatus, KeyGenerator keyGenerator) { + + Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath()); + HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, + table, partitionPath, FSUtils.createNewFileIdPfx(), table.getSparkTaskContextSupplier()); + Schema avroSchema = null; + try { + ParquetMetadata readFooter = ParquetFileReader.readFooter(table.getHadoopConf(), sourceFilePath, + ParquetMetadataConverter.NO_FILTER); + MessageType parquetSchema = readFooter.getFileMetaData().getSchema(); + avroSchema = new AvroSchemaConverter().convert(parquetSchema); + Schema recordKeySchema = HoodieAvroUtils.generateProjectionSchema(avroSchema, + keyGenerator.getRecordKeyFieldNames()); + LOG.info("Schema to be used for reading record Keys :" + recordKeySchema); + AvroReadSupport.setAvroReadSchema(table.getHadoopConf(), recordKeySchema); + AvroReadSupport.setRequestedProjection(table.getHadoopConf(), recordKeySchema); + + BoundedInMemoryExecutor wrapper = null; + try (ParquetReader reader = + AvroParquetReader.builder(sourceFilePath).withConf(table.getHadoopConf()).build()) { + wrapper = new SparkBoundedInMemoryExecutor(config, + new ParquetReaderIterator(reader), new BootstrapRecordConsumer(bootstrapHandle), inp -> { + String recKey = keyGenerator.getKey(inp).getRecordKey(); + GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA); + gr.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recKey); + BootstrapRecordPayload payload = new BootstrapRecordPayload(gr); + HoodieRecord rec = new HoodieRecord(new HoodieKey(recKey, partitionPath), payload); + return rec; + }); + wrapper.execute(); + } catch (Exception e) { + throw new HoodieException(e); + } finally { + bootstrapHandle.close(); + if (null != wrapper) { + wrapper.shutdownNow(); + } + } + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + BootstrapWriteStatus writeStatus = (BootstrapWriteStatus)bootstrapHandle.getWriteStatus(); + BootstrapFileMapping bootstrapFileMapping = new BootstrapFileMapping( + config.getBootstrapSourceBasePath(), srcPartitionPath, partitionPath, + srcFileStatus, writeStatus.getFileId()); + writeStatus.setBootstrapSourceFileMapping(bootstrapFileMapping); + return writeStatus; + } + + /** + * Return Bootstrap Mode selections for partitions listed and figure out bootstrap Schema. + * @return + * @throws IOException + */ + private Map>>> listAndProcessSourcePartitions() throws IOException { + List>> folders = + BootstrapUtils.getAllLeafFoldersWithFiles(bootstrapSourceFileSystem, + config.getBootstrapSourceBasePath(), path -> { + // TODO: Needs to be abstracted out when supporting different formats + // TODO: Remove hoodieFilter + return path.getName().endsWith(HoodieFileFormat.PARQUET.getFileExtension()); + }); + + LOG.info("Fetching Bootstrap Schema !!"); + BootstrapSchemaProvider sourceSchemaProvider = new BootstrapSchemaProvider(config); + bootstrapSchema = sourceSchemaProvider.getBootstrapSchema(jsc, folders).toString(); + LOG.info("Bootstrap Schema :" + bootstrapSchema); + + BootstrapModeSelector selector = + (BootstrapModeSelector) ReflectionUtils.loadClass(config.getBootstrapModeSelectorClass(), config); + + Map> result = selector.select(folders); + Map> partitionToFiles = folders.stream().collect( + Collectors.toMap(Pair::getKey, Pair::getValue)); + + // Ensure all partitions are accounted for + ValidationUtils.checkArgument(partitionToFiles.keySet().equals( + result.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()))); + + return result.entrySet().stream().map(e -> Pair.of(e.getKey(), e.getValue().stream() + .map(p -> Pair.of(p, partitionToFiles.get(p))).collect(Collectors.toList()))) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + } + + private JavaRDD runMetadataBootstrap(List>> partitions) { + if (null == partitions || partitions.isEmpty()) { + return jsc.emptyRDD(); + } + + TypedProperties properties = new TypedProperties(); + properties.putAll(config.getProps()); + KeyGenerator keyGenerator = (KeyGenerator) ReflectionUtils.loadClass(config.getBootstrapKeyGeneratorClass(), + properties); + BootstrapPartitionPathTranslator translator = (BootstrapPartitionPathTranslator) ReflectionUtils.loadClass( + config.getBootstrapPartitionPathTranslatorClass(), properties); + + List>> bootstrapPaths = partitions.stream() + .flatMap(p -> { + String translatedPartitionPath = translator.getBootstrapTranslatedPath(p.getKey()); + return p.getValue().stream().map(f -> Pair.of(p.getKey(), Pair.of(translatedPartitionPath, f))); + }) + .collect(Collectors.toList()); + + return jsc.parallelize(bootstrapPaths, config.getBootstrapParallelism()) + .map(partitionFsPair -> handleMetadataBootstrap(partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(), + partitionFsPair.getRight().getRight(), keyGenerator)); + } + + //TODO: Once we decouple commit protocol, we should change the class hierarchy to avoid doing this. + @Override + protected Partitioner getUpsertPartitioner(WorkloadProfile profile) { + throw new UnsupportedOperationException("Should not called in bootstrap code path"); + } + + @Override + protected Partitioner getInsertPartitioner(WorkloadProfile profile) { + throw new UnsupportedOperationException("Should not called in bootstrap code path"); + } + + @Override + protected Iterator> handleInsert(String idPfx, Iterator> recordItr) { + throw new UnsupportedOperationException("Should not called in bootstrap code path"); + } + + @Override + protected Iterator> handleUpdate(String partitionPath, String fileId, Iterator> recordItr) { + throw new UnsupportedOperationException("Should not called in bootstrap code path"); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapDeltaCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapDeltaCommitActionExecutor.java new file mode 100644 index 0000000000000..08760cc3d272f --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapDeltaCommitActionExecutor.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.bootstrap; + +import java.util.Map; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.commit.CommitActionExecutor; +import org.apache.hudi.table.action.deltacommit.BulkInsertDeltaCommitActionExecutor; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +public class BootstrapDeltaCommitActionExecutor> + extends BootstrapCommitActionExecutor { + + public BootstrapDeltaCommitActionExecutor(JavaSparkContext jsc, + HoodieWriteConfig config, HoodieTable table, + Option> extraMetadata) { + super(jsc, config, table, extraMetadata); + } + + protected CommitActionExecutor getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { + return new BulkInsertDeltaCommitActionExecutor(jsc, new HoodieWriteConfig.Builder().withProps(config.getProps()) + .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, + inputRecordsRDD, extraMetadata); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java new file mode 100644 index 0000000000000..7ee240d96c5ce --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.bootstrap; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.HoodieBootstrapHandle; + +import java.io.IOException; + +/** + * Consumer that dequeues records from queue and sends to Merge Handle for writing. + */ +public class BootstrapRecordConsumer extends BoundedInMemoryQueueConsumer { + + private final HoodieBootstrapHandle bootstrapHandle; + + public BootstrapRecordConsumer(HoodieBootstrapHandle bootstrapHandle) { + this.bootstrapHandle = bootstrapHandle; + } + + @Override + protected void consumeOneRecord(HoodieRecord record) { + try { + bootstrapHandle.write(record, record.getData().getInsertValue(bootstrapHandle.getWriterSchemaWithMetafields())); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } + + @Override + protected void finish() {} + + @Override + protected Void getResult() { + return null; + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java new file mode 100644 index 0000000000000..67d13651ad2e6 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.bootstrap; + +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.OptionalInt; +import java.util.stream.Collectors; + +public class BootstrapUtils { + + /** + * Returns leaf folders with files under a path. + * @param fs File System + * @param basePathStr Base Path to look for leaf folders + * @param filePathFilter Filters to skip directories/paths + * @return list of partition paths with files under them. + * @throws IOException + */ + public static List>> getAllLeafFoldersWithFiles(FileSystem fs, String basePathStr, + PathFilter filePathFilter) throws IOException { + final Path basePath = new Path(basePathStr); + final Map> levelToPartitions = new HashMap<>(); + final Map> partitionToFiles = new HashMap<>(); + FSUtils.processFiles(fs, basePathStr, (status) -> { + if (status.isFile() && filePathFilter.accept(status.getPath())) { + String relativePath = FSUtils.getRelativePartitionPath(basePath, status.getPath().getParent()); + List statusList = partitionToFiles.get(relativePath); + if (null == statusList) { + Integer level = (int) relativePath.chars().filter(ch -> ch == '/').count(); + List dirs = levelToPartitions.get(level); + if (null == dirs) { + dirs = new ArrayList<>(); + levelToPartitions.put(level, dirs); + } + dirs.add(relativePath); + statusList = new ArrayList<>(); + partitionToFiles.put(relativePath, statusList); + } + statusList.add(FileStatusUtils.fromFileStatus(status)); + } + return true; + }, true); + OptionalInt maxLevelOpt = levelToPartitions.keySet().stream().mapToInt(x -> x).max(); + int maxLevel = maxLevelOpt.orElse(-1); + return maxLevel >= 0 ? levelToPartitions.get(maxLevel).stream() + .map(d -> Pair.of(d, partitionToFiles.get(d))).collect(Collectors.toList()) : new ArrayList<>(); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java new file mode 100644 index 0000000000000..4e6167ecb1eb7 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.bootstrap; + +import org.apache.hudi.common.util.Option; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +/** + * Write Result for a bootstrap operation. + */ +public class HoodieBootstrapWriteMetadata { + + private final Option metadataBootstrapResult; + private final Option fullBootstrapResult; + + public HoodieBootstrapWriteMetadata(Option metadataBootstrapResult, + Option fullBootstrapResult) { + this.metadataBootstrapResult = metadataBootstrapResult; + this.fullBootstrapResult = fullBootstrapResult; + } + + public Option getMetadataBootstrapResult() { + return metadataBootstrapResult; + } + + public Option getFullBootstrapResult() { + return fullBootstrapResult; + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 40185c69d35d1..a8f4341f7e0b3 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -18,6 +18,13 @@ package org.apache.hudi.table.action.commit; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.SparkConfigUtils; @@ -38,43 +45,31 @@ import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.WorkloadStat; import org.apache.hudi.table.action.BaseActionExecutor; - import org.apache.hudi.table.action.HoodieWriteMetadata; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.storage.StorageLevel; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.time.Instant; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - import scala.Tuple2; -public abstract class BaseCommitActionExecutor> - extends BaseActionExecutor { +public abstract class BaseCommitActionExecutor, R> + extends BaseActionExecutor { private static final Logger LOG = LogManager.getLogger(BaseCommitActionExecutor.class); + protected final Option> extraMetadata; private final WriteOperationType operationType; protected final SparkTaskContextSupplier sparkTaskContextSupplier = new SparkTaskContextSupplier(); - public BaseCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, - HoodieTable table, String instantTime, WriteOperationType operationType) { - this(jsc, config, table, instantTime, operationType, null); - } - public BaseCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, String instantTime, WriteOperationType operationType, - JavaRDD> inputRecordsRDD) { + Option> extraMetadata) { super(jsc, config, table, instantTime); this.operationType = operationType; + this.extraMetadata = extraMetadata; } public HoodieWriteMetadata execute(JavaRDD> inputRecordsRDD) { @@ -173,13 +168,17 @@ protected void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, protected void commitOnAutoCommit(HoodieWriteMetadata result) { if (config.shouldAutoCommit()) { LOG.info("Auto commit enabled: Committing " + instantTime); - commit(Option.empty(), result); + commit(extraMetadata, result); } else { LOG.info("Auto commit disabled for " + instantTime); } } - private void commit(Option> extraMetadata, HoodieWriteMetadata result) { + protected void commit(Option> extraMetadata, HoodieWriteMetadata result) { + commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect()); + } + + protected void commit(Option> extraMetadata, HoodieWriteMetadata result, List stats) { String actionType = table.getMetaClient().getCommitActionType(); LOG.info("Committing " + instantTime + ", action Type " + actionType); // Create a Hoodie table which encapsulated the commits and files visible @@ -189,7 +188,6 @@ private void commit(Option> extraMetadata, HoodieWriteMetada HoodieCommitMetadata metadata = new HoodieCommitMetadata(); result.setCommitted(true); - List stats = result.getWriteStatuses().map(WriteStatus::getStat).collect(); stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat)); result.setWriteStats(stats); @@ -200,7 +198,7 @@ private void commit(Option> extraMetadata, HoodieWriteMetada if (extraMetadata.isPresent()) { extraMetadata.get().forEach(metadata::addMetadata); } - metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, config.getSchema()); + metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit()); metadata.setOperationType(operationType); try { @@ -229,6 +227,13 @@ protected void finalizeWrite(String instantTime, List stats, Ho } } + /** + * By default, return the writer schema in Write Config for storing in commit. + */ + protected String getSchemaToStoreInCommit() { + return config.getSchema(); + } + protected boolean isWorkloadProfileNeeded() { return true; } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java index ee93f06a81f52..162ae29845fe2 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.action.commit; +import java.util.Map; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -31,30 +32,34 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -public class BulkInsertCommitActionExecutor> - extends CommitActionExecutor { +public class BulkInsertCommitActionExecutor> extends CommitActionExecutor { private final JavaRDD> inputRecordsRDD; - private final Option userDefinedBulkInsertPartitioner; + private final Option> bulkInsertPartitioner; - public BulkInsertCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option userDefinedBulkInsertPartitioner) { - super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT); + public BulkInsertCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD, + Option> bulkInsertPartitioner) { + this(jsc, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); + } + + public BulkInsertCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD, + Option> bulkInsertPartitioner, + Option> extraMetadata) { + super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); this.inputRecordsRDD = inputRecordsRDD; - this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner; + this.bulkInsertPartitioner = bulkInsertPartitioner; } @Override public HoodieWriteMetadata execute() { try { return BulkInsertHelper.bulkInsert(inputRecordsRDD, instantTime, (HoodieTable) table, config, - this, true, userDefinedBulkInsertPartitioner); + this, true, bulkInsertPartitioner); + } catch (HoodieInsertException ie) { + throw ie; } catch (Throwable e) { - if (e instanceof HoodieInsertException) { - throw e; - } throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java index eb38f076db39c..4683c8218d782 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java @@ -39,11 +39,10 @@ public class BulkInsertHelper> { - public static > HoodieWriteMetadata bulkInsert( - JavaRDD> inputRecords, String instantTime, - HoodieTable table, HoodieWriteConfig config, - CommitActionExecutor executor, boolean performDedupe, - Option userDefinedBulkInsertPartitioner) { + public static > HoodieWriteMetadata bulkInsert(JavaRDD> inputRecords, String instantTime, + HoodieTable table, HoodieWriteConfig config, + CommitActionExecutor executor, boolean performDedupe, + Option> userDefinedBulkInsertPartitioner) { HoodieWriteMetadata result = new HoodieWriteMetadata(); // De-dupe/merge if needed diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java index 0f784819925bc..f63d06e95c469 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java @@ -35,12 +35,12 @@ public class BulkInsertPreppedCommitActionExecutor { private final JavaRDD> preppedInputRecordRdd; - private final Option userDefinedBulkInsertPartitioner; + private final Option> userDefinedBulkInsertPartitioner; public BulkInsertPreppedCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, String instantTime, JavaRDD> preppedInputRecordRdd, - Option userDefinedBulkInsertPartitioner) { + Option> userDefinedBulkInsertPartitioner) { super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT); this.preppedInputRecordRdd = preppedInputRecordRdd; this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner; diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/CommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/CommitActionExecutor.java index c07d4c97781c6..fc721ec368994 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/CommitActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/CommitActionExecutor.java @@ -23,21 +23,15 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; -import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.execution.LazyInsertIterable; -import org.apache.hudi.execution.SparkBoundedInMemoryExecutor; import org.apache.hudi.io.HoodieMergeHandle; -import org.apache.hudi.io.storage.HoodieFileReader; -import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; @@ -50,14 +44,19 @@ import java.util.Map; public abstract class CommitActionExecutor> - extends BaseCommitActionExecutor { + extends BaseCommitActionExecutor { private static final Logger LOG = LogManager.getLogger(CommitActionExecutor.class); - public CommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, WriteOperationType operationType) { - super(jsc, config, table, instantTime, operationType); + public CommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType) { + this(jsc, config, table, instantTime, operationType, Option.empty()); + } + + public CommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType, + Option> extraMetadata) { + super(jsc, config, table, instantTime, operationType, extraMetadata); } @Override @@ -87,22 +86,7 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups throw new HoodieUpsertException( "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); } else { - BoundedInMemoryExecutor wrapper = null; - try { - HoodieFileReader storageReader = - HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), upsertHandle.getOldFilePath()); - wrapper = - new SparkBoundedInMemoryExecutor(config, storageReader.getRecordIterator(upsertHandle.getWriterSchema()), - new UpdateHandler(upsertHandle), x -> x); - wrapper.execute(); - } catch (Exception e) { - throw new HoodieException(e); - } finally { - upsertHandle.close(); - if (null != wrapper) { - wrapper.shutdownNow(); - } - } + MergeHelper.runMerge(table, upsertHandle); } // TODO(vc): This needs to be revisited @@ -147,29 +131,4 @@ public Partitioner getUpsertPartitioner(WorkloadProfile profile) { public Partitioner getInsertPartitioner(WorkloadProfile profile) { return getUpsertPartitioner(profile); } - - /** - * Consumer that dequeues records from queue and sends to Merge Handle. - */ - private static class UpdateHandler extends BoundedInMemoryQueueConsumer { - - private final HoodieMergeHandle upsertHandle; - - private UpdateHandler(HoodieMergeHandle upsertHandle) { - this.upsertHandle = upsertHandle; - } - - @Override - protected void consumeOneRecord(GenericRecord record) { - upsertHandle.write(record); - } - - @Override - protected void finish() {} - - @Override - protected Void getResult() { - return null; - } - } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java new file mode 100644 index 0000000000000..4daa5c61fa96a --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.commit; + +import java.io.ByteArrayOutputStream; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.EncoderFactory; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.client.utils.MergingIterator; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.execution.SparkBoundedInMemoryExecutor; +import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.table.HoodieTable; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; +import java.util.Iterator; + +/** + * Helper to read records from previous version of parquet and run Merge. + */ +public class MergeHelper { + + /** + * Read records from previous version of base file and merge. + * @param table Hoodie Table + * @param upsertHandle Merge Handle + * @param + * @throws IOException in case of error + */ + public static > void runMerge(HoodieTable table, HoodieMergeHandle upsertHandle) throws IOException { + final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation(); + Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf()); + HoodieBaseFile baseFile = upsertHandle.baseFileForMerge(); + + final GenericDatumWriter gWriter; + final GenericDatumReader gReader; + Schema readSchema; + if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) { + readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), upsertHandle.getOldFilePath()).getSchema(); + gWriter = new GenericDatumWriter<>(readSchema); + gReader = new GenericDatumReader<>(readSchema, upsertHandle.getWriterSchemaWithMetafields()); + } else { + gReader = null; + gWriter = null; + readSchema = upsertHandle.getWriterSchemaWithMetafields(); + } + + BoundedInMemoryExecutor wrapper = null; + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, upsertHandle.getOldFilePath()); + try { + final Iterator readerIterator; + if (baseFile.getBootstrapBaseFile().isPresent()) { + readerIterator = getMergingIterator(table, upsertHandle, baseFile, reader, readSchema, externalSchemaTransformation); + } else { + readerIterator = reader.getRecordIterator(readSchema); + } + + ThreadLocal encoderCache = new ThreadLocal<>(); + ThreadLocal decoderCache = new ThreadLocal<>(); + wrapper = new SparkBoundedInMemoryExecutor(table.getConfig(), readerIterator, + new UpdateHandler(upsertHandle), record -> { + if (!externalSchemaTransformation) { + return record; + } + return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) record); + }); + wrapper.execute(); + } catch (Exception e) { + throw new HoodieException(e); + } finally { + if (reader != null) { + reader.close(); + } + upsertHandle.close(); + if (null != wrapper) { + wrapper.shutdownNow(); + } + } + } + + private static GenericRecord transformRecordBasedOnNewSchema(GenericDatumReader gReader, GenericDatumWriter gWriter, + ThreadLocal encoderCache, ThreadLocal decoderCache, + GenericRecord gRec) { + ByteArrayOutputStream inStream = null; + try { + inStream = new ByteArrayOutputStream(); + BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(inStream, encoderCache.get()); + encoderCache.set(encoder); + gWriter.write(gRec, encoder); + encoder.flush(); + + BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inStream.toByteArray(), decoderCache.get()); + decoderCache.set(decoder); + GenericRecord transformedRec = gReader.read(null, decoder); + return transformedRec; + } catch (IOException e) { + throw new HoodieException(e); + } finally { + try { + inStream.close(); + } catch (IOException ioe) { + throw new HoodieException(ioe.getMessage(), ioe); + } + } + } + + /** + * Create Parquet record iterator that provides a stitched view of record read from skeleton and bootstrap file. + * Skeleton file is a representation of the bootstrap file inside the table, with just the bare bone fields needed + * for indexing, writing and other functionality. + * + */ + private static > Iterator getMergingIterator(HoodieTable table, HoodieMergeHandle mergeHandle, + HoodieBaseFile baseFile, HoodieFileReader reader, + Schema readSchema, boolean externalSchemaTransformation) throws IOException { + Path externalFilePath = new Path(baseFile.getBootstrapBaseFile().get().getPath()); + Configuration bootstrapFileConfig = new Configuration(table.getHadoopConf()); + HoodieFileReader bootstrapReader = HoodieFileReaderFactory.getFileReader(bootstrapFileConfig, externalFilePath); + Schema bootstrapReadSchema; + if (externalSchemaTransformation) { + bootstrapReadSchema = bootstrapReader.getSchema(); + } else { + bootstrapReadSchema = mergeHandle.getWriterSchema(); + } + + return new MergingIterator<>(reader.getRecordIterator(readSchema), bootstrapReader.getRecordIterator(bootstrapReadSchema), + (inputRecordPair) -> HoodieAvroUtils.stitchRecords(inputRecordPair.getLeft(), inputRecordPair.getRight(), mergeHandle.getWriterSchemaWithMetafields())); + } + + /** + * Consumer that dequeues records from queue and sends to Merge Handle. + */ + private static class UpdateHandler extends BoundedInMemoryQueueConsumer { + + private final HoodieMergeHandle upsertHandle; + + private UpdateHandler(HoodieMergeHandle upsertHandle) { + this.upsertHandle = upsertHandle; + } + + @Override + protected void consumeOneRecord(GenericRecord record) { + upsertHandle.write(record); + } + + @Override + protected void finish() {} + + @Override + protected Void getResult() { + return null; + } + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java index 6d88e5cc15aaa..0d7f6bee8978f 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java @@ -56,22 +56,23 @@ public ScheduleCompactionActionExecutor(JavaSparkContext jsc, private HoodieCompactionPlan scheduleCompaction() { LOG.info("Checking if compaction needs to be run on " + config.getBasePath()); - Option lastCompaction = table.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant(); - String deltaCommitsSinceTs = "0"; + Option lastCompaction = table.getActiveTimeline().getCommitTimeline() + .filterCompletedInstants().lastInstant(); + String lastCompactionTs = "0"; if (lastCompaction.isPresent()) { - deltaCommitsSinceTs = lastCompaction.get().getTimestamp(); + lastCompactionTs = lastCompaction.get().getTimestamp(); } int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline() - .findInstantsAfter(deltaCommitsSinceTs, Integer.MAX_VALUE).countInstants(); + .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants(); if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) { LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction - + " delta commits was found since last compaction " + deltaCommitsSinceTs + ". Waiting for " + + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for " + config.getInlineCompactDeltaCommitMax()); return new HoodieCompactionPlan(); } - LOG.info("Compacting merge on read table " + config.getBasePath()); + LOG.info("Generating compaction plan for merge on read table " + config.getBasePath()); HoodieMergeOnReadTableCompactor compactor = new HoodieMergeOnReadTableCompactor(); try { return compactor.generateCompactionPlan(jsc, table, config, instantTime, diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertDeltaCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertDeltaCommitActionExecutor.java index 61eb6128b05d8..01ff1faa4b3fd 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertDeltaCommitActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertDeltaCommitActionExecutor.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.action.deltacommit; +import java.util.Map; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -36,13 +37,19 @@ public class BulkInsertDeltaCommitActionExecutor { private final JavaRDD> inputRecordsRDD; - private final Option bulkInsertPartitioner; + private final Option> bulkInsertPartitioner; - public BulkInsertDeltaCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option bulkInsertPartitioner) { - super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT); + public BulkInsertDeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD, + Option> bulkInsertPartitioner) { + this(jsc, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); + } + + public BulkInsertDeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD, + Option> bulkInsertPartitioner, + Option> extraMetadata) { + super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); this.inputRecordsRDD = inputRecordsRDD; this.bulkInsertPartitioner = bulkInsertPartitioner; } @@ -52,10 +59,9 @@ public HoodieWriteMetadata execute() { try { return BulkInsertHelper.bulkInsert(inputRecordsRDD, instantTime, (HoodieTable) table, config, this, true, bulkInsertPartitioner); + } catch (HoodieInsertException ie) { + throw ie; } catch (Throwable e) { - if (e instanceof HoodieInsertException) { - throw e; - } throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e); } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertPreppedDeltaCommitActionExecutor.java index 7ebd4320fb146..7c956008b9ec6 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertPreppedDeltaCommitActionExecutor.java @@ -36,12 +36,12 @@ public class BulkInsertPreppedDeltaCommitActionExecutor { private final JavaRDD> preppedInputRecordRdd; - private final Option bulkInsertPartitioner; + private final Option> bulkInsertPartitioner; public BulkInsertPreppedDeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, String instantTime, JavaRDD> preppedInputRecordRdd, - Option bulkInsertPartitioner) { + Option> bulkInsertPartitioner) { super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT); this.preppedInputRecordRdd = preppedInputRecordRdd; this.bulkInsertPartitioner = bulkInsertPartitioner; diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeltaCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeltaCommitActionExecutor.java index 9429069d88047..f616a00b04e2e 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeltaCommitActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeltaCommitActionExecutor.java @@ -18,10 +18,12 @@ package org.apache.hudi.table.action.deltacommit; +import java.util.Map; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.execution.LazyInsertIterable; @@ -48,10 +50,15 @@ public abstract class DeltaCommitActionExecutor // UpsertPartitioner for MergeOnRead table type private UpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner; - public DeltaCommitActionExecutor(JavaSparkContext jsc, - HoodieWriteConfig config, HoodieTable table, - String instantTime, WriteOperationType operationType) { - super(jsc, config, table, instantTime, operationType); + public DeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType) { + this(jsc, config, table, instantTime, operationType, Option.empty()); + } + + public DeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table, + String instantTime, WriteOperationType operationType, + Option> extraMetadata) { + super(jsc, config, table, instantTime, operationType, extraMetadata); } @Override diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index 90b9bb387168b..2680690805c27 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -224,4 +225,11 @@ protected void deleteInflightAndRequestedInstant(boolean deleteInstant, LOG.warn("Rollback finished without deleting inflight instant file. Instant=" + instantToBeDeleted); } } + + protected void dropBootstrapIndexIfNeeded(HoodieInstant instantToRollback) { + if (HoodieTimeline.compareTimestamps(instantToRollback.getTimestamp(), HoodieTimeline.EQUALS, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)) { + LOG.info("Dropping bootstrap index as metadata bootstrap commit is getting rolled back !!"); + BootstrapIndex.getBootstrapIndex(table.getMetaClient()).dropIndex(); + } + } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java index f2b69786afe99..eb8def01fb9da 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java @@ -82,6 +82,9 @@ protected List executeRollback() { LOG.info("Clean out all base files generated for commit: " + resolvedInstant); stats = getRollbackStrategy().execute(resolvedInstant); } + + dropBootstrapIndexIfNeeded(instantToRollback); + // Delete Inflight instant if enabled deleteInflightAndRequestedInstant(deleteInstants, activeTimeline, resolvedInstant); LOG.info("Time(in ms) taken to finish rollback " + rollbackTimer.endTimer()); diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java index d11b7added27d..c29cefaad772c 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java @@ -98,6 +98,8 @@ protected List executeRollback() throws IOException { allRollbackStats = getRollbackStrategy().execute(resolvedInstant); } + dropBootstrapIndexIfNeeded(resolvedInstant); + // Delete Inflight instants if enabled deleteInflightAndRequestedInstant(deleteInstants, table.getActiveTimeline(), resolvedInstant); LOG.info("Time(in ms) taken to finish rollback " + rollbackTimer.endTimer()); diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java index b1d1aa8c29655..7eea3ae3d97cd 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java @@ -477,8 +477,8 @@ private void testUpsertsUpdatePartitionPath(IndexType indexType, HoodieWriteConf .withBloomIndexUpdatePartitionPath(true) .withGlobalSimpleIndexUpdatePartitionPath(true) .build()).withTimelineLayoutVersion(VERSION_0).build(); - HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(), - metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(), + HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), + metaClient.getTableType(), metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_0); HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false); diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java b/hudi-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java index c6806b46a4110..2287979e4aea2 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java +++ b/hudi-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java @@ -122,7 +122,7 @@ public void testSchemaEvolutionOnUpdate() throws Exception { HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config2, "101", table2, updateRecords.iterator(), record1.getPartitionPath(), fileId, supplier); Configuration conf = new Configuration(); - AvroReadSupport.setAvroReadSchema(conf, mergeHandle.getWriterSchema()); + AvroReadSupport.setAvroReadSchema(conf, mergeHandle.getWriterSchemaWithMetafields()); List oldRecords = ParquetUtils.readAvroRecords(conf, new Path(config2.getBasePath() + "/" + insertResult.getStat().getPath())); for (GenericRecord rec : oldRecords) { diff --git a/hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java b/hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java new file mode 100644 index 0000000000000..3499a8887daf3 --- /dev/null +++ b/hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap; + +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.selector.BootstrapRegexModeSelector; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieBootstrapConfig; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestBootstrapRegexModeSelector { + + private HoodieWriteConfig getConfig(String regex, BootstrapMode selectedMode) { + return HoodieWriteConfig.newBuilder().withPath("") + .withBootstrapConfig(HoodieBootstrapConfig.newBuilder() + .withBootstrapModeSelectorRegex(regex) + .withBootstrapModeForRegexMatch(selectedMode).build()) + .forTable("test-trip-table").build(); + } + + @Test + public void testModeSelector() { + List partitionPaths = Arrays.asList("2020/05/01", "2020/05/02", "2020/05/10", "2020/05/11"); + List>> input = partitionPaths.stream() + .map(p -> Pair.>of(p, new ArrayList<>())).collect(Collectors.toList()); + String regex = "2020/05/1[0-9]"; + BootstrapRegexModeSelector regexModeSelector = new BootstrapRegexModeSelector(getConfig(regex, + BootstrapMode.FULL_RECORD)); + + Map> result = regexModeSelector.select(input); + assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/01")); + assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/02")); + assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/10")); + assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/11")); + assertEquals(2, result.get(BootstrapMode.METADATA_ONLY).size()); + assertEquals(2, result.get(BootstrapMode.FULL_RECORD).size()); + + regexModeSelector = new BootstrapRegexModeSelector(getConfig(regex, + BootstrapMode.METADATA_ONLY)); + result = regexModeSelector.select(input); + assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/01")); + assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/02")); + assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/10")); + assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/11")); + assertEquals(2, result.get(BootstrapMode.METADATA_ONLY).size()); + assertEquals(2, result.get(BootstrapMode.FULL_RECORD).size()); + } +} diff --git a/hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java b/hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java new file mode 100644 index 0000000000000..d1425f6fab707 --- /dev/null +++ b/hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.bootstrap; + +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector; +import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector; +import org.apache.hudi.client.bootstrap.selector.UniformBootstrapModeSelector; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestUniformBootstrapModeSelector { + + @Test + public void testFullBootstrapModeSelector() { + + FullRecordBootstrapModeSelector modeSelector = new FullRecordBootstrapModeSelector( + HoodieWriteConfig.newBuilder().withPath("").build()); + testModeSelector(modeSelector, BootstrapMode.FULL_RECORD); + } + + @Test + public void testMetadataOnlyBootstrapModeSelector() { + MetadataOnlyBootstrapModeSelector modeSelector = new MetadataOnlyBootstrapModeSelector( + HoodieWriteConfig.newBuilder().withPath("").build()); + testModeSelector(modeSelector, BootstrapMode.METADATA_ONLY); + } + + private void testModeSelector(UniformBootstrapModeSelector modeSelector, BootstrapMode mode) { + List partitionPaths = Arrays.asList("2020/05/01", "2020/05/02", "2020/05/10", "2020/05/11"); + List>> input = partitionPaths.stream() + .map(p -> Pair.>of(p, new ArrayList<>())).collect(Collectors.toList()); + Map> result = modeSelector.select(input); + assertTrue(result.get(mode).contains("2020/05/01")); + assertTrue(result.get(mode).contains("2020/05/02")); + assertTrue(result.get(mode).contains("2020/05/10")); + assertTrue(result.get(mode).contains("2020/05/11")); + assertEquals(4, result.get(mode).size()); + } +} diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java b/hudi-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java new file mode 100644 index 0000000000000..7303bee1ce350 --- /dev/null +++ b/hudi-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.bootstrap; + +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.testutils.HoodieClientTestBase; + +import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestBootstrapUtils extends HoodieClientTestBase { + + @Test + public void testAllLeafFoldersWithFiles() throws IOException { + // All directories including marker dirs. + List folders = Arrays.asList("2016/04/15", "2016/05/16", "2016/05/17"); + folders.forEach(f -> { + try { + metaClient.getFs().mkdirs(new Path(new Path(basePath), f)); + } catch (IOException e) { + throw new HoodieException(e); + } + }); + + // Files inside partitions and marker directories + List files = Arrays.asList( + "2016/04/15/1_1-0-1_20190528120000.parquet", + "2016/04/15/2_1-0-1_20190528120000.parquet", + "2016/05/16/3_1-0-1_20190528120000.parquet", + "2016/05/16/4_1-0-1_20190528120000.parquet", + "2016/04/17/5_1-0-1_20190528120000.parquet", + "2016/04/17/6_1-0-1_20190528120000.parquet"); + + files.forEach(f -> { + try { + metaClient.getFs().create(new Path(new Path(basePath), f)); + } catch (IOException e) { + throw new HoodieException(e); + } + }); + + List>> collected = + BootstrapUtils.getAllLeafFoldersWithFiles(metaClient.getFs(), basePath, (status) -> { + return true; + }); + assertEquals(3, collected.size()); + collected.stream().forEach(k -> { + assertEquals(2, k.getRight().size()); + }); + + // Simulate reading from un-partitioned dataset + collected = + BootstrapUtils.getAllLeafFoldersWithFiles(metaClient.getFs(), basePath + "/" + folders.get(0), (status) -> { + return true; + }); + assertEquals(1, collected.size()); + collected.stream().forEach(k -> { + assertEquals(2, k.getRight().size()); + }); + } +} diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java b/hudi-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java index 44899c33e1bc8..d140b1183aacf 100644 --- a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java +++ b/hudi-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.compact.strategy; import org.apache.hudi.avro.model.HoodieCompactionOperation; +import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; @@ -259,7 +260,9 @@ private List createCompactionOperations(HoodieWriteCo operations.add(new HoodieCompactionOperation(df.getCommitTime(), logFiles.stream().map(s -> s.getPath().toString()).collect(Collectors.toList()), df.getPath(), df.getFileId(), partitionPath, - config.getCompactionStrategy().captureMetrics(config, Option.of(df), partitionPath, logFiles))); + config.getCompactionStrategy().captureMetrics(config, Option.of(df), partitionPath, logFiles), + df.getBootstrapBaseFile().map(BaseFile::getPath).orElse(null)) + ); }); return operations; } diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java b/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java index 7ee3dc9339903..8104ef7744fce 100644 --- a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java +++ b/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java @@ -22,14 +22,17 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.ql.io.IOConstants; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.io.Writable; @@ -48,22 +51,39 @@ * Utility methods to aid in testing MergeOnRead (workaround for HoodieReadClient for MOR). */ public class HoodieMergeOnReadTestUtils { + public static List getRecordsUsingInputFormat(Configuration conf, List inputPaths, String basePath) { return getRecordsUsingInputFormat(conf, inputPaths, basePath, new JobConf(conf), true); } public static List getRecordsUsingInputFormat(Configuration conf, List inputPaths, - String basePath, - JobConf jobConf, - boolean realtime) { + String basePath, JobConf jobConf, boolean realtime) { + Schema schema = new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA); + return getRecordsUsingInputFormat(conf, inputPaths, basePath, jobConf, realtime, schema, + HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); + } + + public static List getRecordsUsingInputFormat(Configuration conf, List inputPaths, String basePath, JobConf jobConf, boolean realtime, Schema rawSchema, + String rawHiveColumnTypes, boolean projectCols, List projectedColumns) { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf, basePath); - FileInputFormat inputFormat = HoodieInputFormatUtils.getInputFormat(metaClient.getTableConfig().getBaseFileFormat(), - realtime, jobConf); + FileInputFormat inputFormat = HoodieInputFormatUtils.getInputFormat(metaClient.getTableConfig().getBaseFileFormat(), realtime, jobConf); + + Schema schema = HoodieAvroUtils.addMetadataFields(rawSchema); + String hiveColumnTypes = HoodieAvroUtils.addMetadataColumnTypes(rawHiveColumnTypes); + setPropsForInputFormat(inputFormat, jobConf, schema, hiveColumnTypes, projectCols, projectedColumns); + final List fields; + if (projectCols) { + fields = schema.getFields().stream().filter(f -> projectedColumns.contains(f.name())) + .collect(Collectors.toList()); + } else { + fields = schema.getFields(); + } + final Schema projectedSchema = Schema.createRecord(fields.stream() + .map(f -> new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal())) + .collect(Collectors.toList())); - Schema schema = HoodieAvroUtils.addMetadataFields( - new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)); - setPropsForInputFormat(inputFormat, jobConf, schema, basePath); return inputPaths.stream().map(path -> { setInputPath(jobConf, path); List records = new ArrayList<>(); @@ -71,17 +91,18 @@ public static List getRecordsUsingInputFormat(Configuration conf, List splits = Arrays.asList(inputFormat.getSplits(jobConf, 1)); for (InputSplit split : splits) { RecordReader recordReader = inputFormat.getRecordReader(split, jobConf, null); - Void key = (Void) recordReader.createKey(); + Object key = recordReader.createKey(); ArrayWritable writable = (ArrayWritable) recordReader.createValue(); while (recordReader.next(key, writable)) { - GenericRecordBuilder newRecord = new GenericRecordBuilder(schema); + GenericRecordBuilder newRecord = new GenericRecordBuilder(projectedSchema); // writable returns an array with [field1, field2, _hoodie_commit_time, // _hoodie_commit_seqno] Writable[] values = writable.get(); - assert schema.getFields().size() <= values.length; - schema.getFields().forEach(field -> { - newRecord.set(field, values[field.pos()]); - }); + schema.getFields().stream() + .filter(f -> !projectCols || projectedColumns.contains(f.name())) + .map(f -> Pair.of(projectedSchema.getFields().stream() + .filter(p -> f.name().equals(p.name())).findFirst().get(), f)) + .forEach(fieldsPair -> newRecord.set(fieldsPair.getKey(), values[fieldsPair.getValue().pos()])); records.add(newRecord.build()); } } @@ -95,29 +116,40 @@ public static List getRecordsUsingInputFormat(Configuration conf, }).orElse(new ArrayList<>()); } - private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf jobConf, Schema schema, - String basePath) { + private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf jobConf, Schema schema, String hiveColumnTypes, boolean projectCols, List projectedCols) { List fields = schema.getFields(); - String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); - String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(",")); - Configuration conf = HoodieTestUtils.getDefaultHadoopConf(); + final List projectedColNames; + if (!projectCols) { + projectedColNames = fields.stream().map(Field::name).collect(Collectors.toList()); + } else { + projectedColNames = projectedCols; + } - String hiveColumnNames = fields.stream().filter(field -> !field.name().equalsIgnoreCase("datestr")) + String names = fields.stream() + .filter(f -> projectedColNames.contains(f.name())) + .map(f -> f.name()).collect(Collectors.joining(",")); + String positions = fields.stream() + .filter(f -> projectedColNames.contains(f.name())) + .map(f -> String.valueOf(f.pos())).collect(Collectors.joining(",")); + String hiveColumnNames = fields.stream() + .filter(field -> !field.name().equalsIgnoreCase("datestr")) .map(Schema.Field::name).collect(Collectors.joining(",")); hiveColumnNames = hiveColumnNames + ",datestr"; - String hiveColumnTypes = HoodieAvroUtils.addMetadataColumnTypes(HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES); - hiveColumnTypes = hiveColumnTypes + ",string"; + Configuration conf = HoodieTestUtils.getDefaultHadoopConf(); + String hiveColumnTypesWithDatestr = hiveColumnTypes + ",string"; jobConf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames); - jobConf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, hiveColumnTypes); + jobConf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, hiveColumnTypesWithDatestr); jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names); - jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions); + jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions); jobConf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "datestr"); conf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames); conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names); - conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions); + conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions); conf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "datestr"); - conf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, hiveColumnTypes); + conf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, hiveColumnTypesWithDatestr); + conf.set(IOConstants.COLUMNS, hiveColumnNames); + conf.get(IOConstants.COLUMNS_TYPES, hiveColumnTypesWithDatestr); // Hoodie Input formats are also configurable Configurable configurable = (Configurable)inputFormat; diff --git a/hudi-client/src/test/resources/log4j-surefire-quiet.properties b/hudi-client/src/test/resources/log4j-surefire-quiet.properties index dc9e35d893b04..2b94ea2903067 100644 --- a/hudi-client/src/test/resources/log4j-surefire-quiet.properties +++ b/hudi-client/src/test/resources/log4j-surefire-quiet.properties @@ -17,6 +17,7 @@ ### log4j.rootLogger=WARN, CONSOLE log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR # CONSOLE is set to be a ConsoleAppender. log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender diff --git a/hudi-client/src/test/resources/log4j-surefire.properties b/hudi-client/src/test/resources/log4j-surefire.properties index b1ccce8f962e0..32af462093ae5 100644 --- a/hudi-client/src/test/resources/log4j-surefire.properties +++ b/hudi-client/src/test/resources/log4j-surefire.properties @@ -18,6 +18,7 @@ log4j.rootLogger=WARN, CONSOLE log4j.logger.org.apache=INFO log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR # A1 is set to be a ConsoleAppender. log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index a05b56920b1a2..d9901dab818ac 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -75,6 +75,12 @@ ${basedir}/src/main/avro/HoodieRollbackMetadata.avsc ${basedir}/src/main/avro/HoodieRestoreMetadata.avsc ${basedir}/src/main/avro/HoodieArchivedMetaEntry.avsc + ${basedir}/src/main/avro/HoodiePath.avsc + ${basedir}/src/main/avro/HoodieFSPermission.avsc + ${basedir}/src/main/avro/HoodieFileStatus.avsc + ${basedir}/src/main/avro/HoodieBootstrapSourceFilePartitionInfo.avsc + ${basedir}/src/main/avro/HoodieBootstrapIndexInfo.avsc + ${basedir}/src/main/avro/HoodieBootstrapMetadata.avsc @@ -197,12 +203,31 @@ ${hbase.version} test + org.apache.hbase hbase-server ${hbase.version} - test + + compile + + + javax.servlet + * + + + org.codehaus.jackson + * + + + org.mortbay.jetty + * + + + tomcat + * + + - diff --git a/hudi-common/src/main/avro/HoodieBootstrapFilePartitionInfo.avsc b/hudi-common/src/main/avro/HoodieBootstrapFilePartitionInfo.avsc new file mode 100644 index 0000000000000..c9f889be03238 --- /dev/null +++ b/hudi-common/src/main/avro/HoodieBootstrapFilePartitionInfo.avsc @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +{ + "namespace":"org.apache.hudi.avro.model", + "type":"record", + "name":"HoodieBootstrapFilePartitionInfo", + "fields":[ + { + "name":"version", + "type":["int", "null"], + "default": 1 + }, + { + "name":"bootstrapPartitionPath", + "type":["null", "string"], + "default" : null + }, + { + "name":"bootstrapFileStatus", + "type":["null", "HoodieFileStatus"], + "default" : null + }, + { + "name":"partitionPath", + "type":["null", "string"], + "default" : null + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieBootstrapIndexInfo.avsc b/hudi-common/src/main/avro/HoodieBootstrapIndexInfo.avsc new file mode 100644 index 0000000000000..c8ae52cfbb658 --- /dev/null +++ b/hudi-common/src/main/avro/HoodieBootstrapIndexInfo.avsc @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +{ + "namespace":"org.apache.hudi.avro.model", + "type":"record", + "name":"HoodieBootstrapIndexInfo", + "fields":[ + { + "name":"version", + "type":["int", "null"], + "default": 1 + }, + { + "name":"bootstrapBasePath", + "type":["null", "string"], + "default" : null + }, + { + "name":"createdTimestamp", + "type":["null", "long"], + "default" : null + }, + { + "name":"numKeys", + "type":["null", "int"], + "default" : null + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieBootstrapPartitionMetadata.avsc b/hudi-common/src/main/avro/HoodieBootstrapPartitionMetadata.avsc new file mode 100644 index 0000000000000..6773b7dc84ef8 --- /dev/null +++ b/hudi-common/src/main/avro/HoodieBootstrapPartitionMetadata.avsc @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +{ + "namespace":"org.apache.hudi.avro.model", + "type":"record", + "name":"HoodieBootstrapPartitionMetadata", + "fields":[ + { + "name":"version", + "type":["int", "null"], + "default": 1 + }, + { + "name":"bootstrapPartitionPath", + "type":["null", "string"], + "default" : null + }, + { + "name":"partitionPath", + "type":["null", "string"], + "default" : null + }, + { + "name":"fileIdToBootstrapFile", + "type":["null", { + "type":"map", + "values": "HoodieFileStatus" + }], + "default": null + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieCompactionOperation.avsc b/hudi-common/src/main/avro/HoodieCompactionOperation.avsc index 0036e5739445f..2095a9518c53e 100644 --- a/hudi-common/src/main/avro/HoodieCompactionOperation.avsc +++ b/hudi-common/src/main/avro/HoodieCompactionOperation.avsc @@ -61,6 +61,11 @@ "values":"double" }], "default": null + }, + { + "name":"bootstrapFilePath", + "type":["null", "string"], + "default": null } ] } diff --git a/hudi-common/src/main/avro/HoodieFSPermission.avsc b/hudi-common/src/main/avro/HoodieFSPermission.avsc new file mode 100644 index 0000000000000..e5893352b3099 --- /dev/null +++ b/hudi-common/src/main/avro/HoodieFSPermission.avsc @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +{ + "namespace":"org.apache.hudi.avro.model", + "type":"record", + "name":"HoodieFSPermission", + "fields":[ + { + "name":"version", + "type":["int", "null"], + "default": 1 + }, + { + "name":"userAction", + "type":[ "null", "string" ], + "default": "null" + }, + { + "name":"groupAction", + "type":[ "null", "string" ], + "default": "null" + }, + { + "name":"otherAction", + "type":[ "null", "string" ], + "default": "null" + }, + { + "name":"stickyBit", + "type":[ "null", "boolean" ], + "default": "null" + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieFileStatus.avsc b/hudi-common/src/main/avro/HoodieFileStatus.avsc new file mode 100644 index 0000000000000..39c12c41e800a --- /dev/null +++ b/hudi-common/src/main/avro/HoodieFileStatus.avsc @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +{ + "namespace":"org.apache.hudi.avro.model", + "type":"record", + "name":"HoodieFileStatus", + "fields":[ + { + "name":"version", + "type":["int", "null"], + "default": 1 + }, + { + "name":"path", + "type":["null", "HoodiePath"], + "default" : null + }, + { + "name":"length", + "type":["null", "long"], + "default" : null + }, + { + "name":"isDir", + "type":["null", "boolean"], + "default" : null + }, + { + "name":"blockReplication", + "type":["null", "int"], + "default" : null + }, + { + "name":"blockSize", + "type":["null", "long"], + "default" : null + }, + { + "name":"modificationTime", + "type":["null", "long"], + "default" : null + }, + { + "name":"accessTime", + "type":["null", "long"], + "default" : null + }, + { + "name":"permission", + "type":["null", "HoodieFSPermission"], + "default" : null + }, + { + "name":"owner", + "type":["null", "string"], + "default" : null + }, + { + "name":"group", + "type":["null", "string"], + "default" : null + }, + { + "name":"symlink", + "type":["null", "HoodiePath"], + "default" : null + } + ] +} diff --git a/hudi-common/src/main/avro/HoodiePath.avsc b/hudi-common/src/main/avro/HoodiePath.avsc new file mode 100644 index 0000000000000..be114adba7dc6 --- /dev/null +++ b/hudi-common/src/main/avro/HoodiePath.avsc @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +{ + "namespace":"org.apache.hudi.avro.model", + "type":"record", + "name":"HoodiePath", + "fields":[ + { + "name":"version", + "type":["int", "null"], + "default": 1 + }, + { + "name":"uri", + "type":["null", "string"], + "default" : null + } + ] +} diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index 38b9d32097196..c6374d7bb215c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -19,7 +19,13 @@ package org.apache.hudi.avro; import org.apache.avro.JsonProperties; +import java.time.LocalDate; +import org.apache.avro.LogicalTypes; +import org.apache.avro.generic.GenericData.Record; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.SchemaCompatabilityException; @@ -64,10 +70,10 @@ public class HoodieAvroUtils { private static ThreadLocal reuseDecoder = ThreadLocal.withInitial(() -> null); // All metadata fields are optional strings. - static final Schema METADATA_FIELD_SCHEMA = + public static final Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING))); - private static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); + public static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); /** * Convert a given avro record to bytes. @@ -251,6 +257,17 @@ public static GenericRecord addCommitMetadataToRecord(GenericRecord record, Stri return record; } + public static GenericRecord stitchRecords(GenericRecord left, GenericRecord right, Schema stitchedSchema) { + GenericRecord result = new Record(stitchedSchema); + for (Schema.Field f : left.getSchema().getFields()) { + result.put(f.name(), left.get(f.name())); + } + for (Schema.Field f : right.getSchema().getFields()) { + result.put(f.name(), right.get(f.name())); + } + return result; + } + /** * Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the old * schema. @@ -326,4 +343,105 @@ public static String decompress(byte[] bytes) { throw new HoodieIOException("IOException while decompressing text", e); } } + + /** + * Generate a reader schema off the provided writeSchema, to just project out the provided columns. + */ + public static Schema generateProjectionSchema(Schema originalSchema, List fieldNames) { + Map schemaFieldsMap = originalSchema.getFields().stream() + .map(r -> Pair.of(r.name().toLowerCase(), r)).collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + List projectedFields = new ArrayList<>(); + for (String fn : fieldNames) { + Schema.Field field = schemaFieldsMap.get(fn.toLowerCase()); + if (field == null) { + throw new HoodieException("Field " + fn + " not found in log schema. Query cannot proceed! " + + "Derived Schema Fields: " + new ArrayList<>(schemaFieldsMap.keySet())); + } else { + projectedFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue())); + } + } + + Schema projectedSchema = Schema.createRecord(originalSchema.getName(), originalSchema.getDoc(), + originalSchema.getNamespace(), originalSchema.isError()); + projectedSchema.setFields(projectedFields); + return projectedSchema; + } + + /** + * Obtain value of the provided field as string, denoted by dot notation. e.g: a.b.c + */ + public static String getNestedFieldValAsString(GenericRecord record, String fieldName, boolean returnNullIfNotFound) { + Object obj = getNestedFieldVal(record, fieldName, returnNullIfNotFound); + return StringUtils.objToString(obj); + } + + /** + * Obtain value of the provided field, denoted by dot notation. e.g: a.b.c + */ + public static Object getNestedFieldVal(GenericRecord record, String fieldName, boolean returnNullIfNotFound) { + String[] parts = fieldName.split("\\."); + GenericRecord valueNode = record; + int i = 0; + for (; i < parts.length; i++) { + String part = parts[i]; + Object val = valueNode.get(part); + if (val == null) { + break; + } + + // return, if last part of name + if (i == parts.length - 1) { + Schema fieldSchema = valueNode.getSchema().getField(part).schema(); + return convertValueForSpecificDataTypes(fieldSchema, val); + } else { + // VC: Need a test here + if (!(val instanceof GenericRecord)) { + throw new HoodieException("Cannot find a record at part value :" + part); + } + valueNode = (GenericRecord) val; + } + } + + if (returnNullIfNotFound) { + return null; + } else { + throw new HoodieException( + fieldName + "(Part -" + parts[i] + ") field not found in record. Acceptable fields were :" + + valueNode.getSchema().getFields().stream().map(Field::name).collect(Collectors.toList())); + } + } + + /** + * This method converts values for fields with certain Avro/Parquet data types that require special handling. + * + * Logical Date Type is converted to actual Date value instead of Epoch Integer which is how it is + * represented/stored in parquet. + * + * @param fieldSchema avro field schema + * @param fieldValue avro field value + * @return field value either converted (for certain data types) or as it is. + */ + private static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object fieldValue) { + if (fieldSchema == null) { + return fieldValue; + } + + if (isLogicalTypeDate(fieldSchema)) { + return LocalDate.ofEpochDay(Long.parseLong(fieldValue.toString())); + } + return fieldValue; + } + + /** + * Given an Avro field schema checks whether the field is of Logical Date Type or not. + * + * @param fieldSchema avro field schema + * @return boolean indicating whether fieldSchema is of Avro's Date Logical Type + */ + private static boolean isLogicalTypeDate(Schema fieldSchema) { + if (fieldSchema.getType() == Schema.Type.UNION) { + return fieldSchema.getTypes().stream().anyMatch(schema -> schema.getLogicalType() == LogicalTypes.date()); + } + return fieldSchema.getLogicalType() == LogicalTypes.date(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/FileStatusUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/FileStatusUtils.java new file mode 100644 index 0000000000000..b33c71d3a86b2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/FileStatusUtils.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.bootstrap; + +import org.apache.hudi.avro.model.HoodieFSPermission; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.avro.model.HoodiePath; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; + +import java.io.IOException; + +/** + * Helper functions around FileStatus and HoodieFileStatus. + */ +public class FileStatusUtils { + + public static Path toPath(HoodiePath path) { + if (null == path) { + return null; + } + return new Path(path.getUri()); + } + + public static HoodiePath fromPath(Path path) { + if (null == path) { + return null; + } + return HoodiePath.newBuilder().setUri(path.toString()).build(); + } + + public static FsPermission toFSPermission(HoodieFSPermission fsPermission) { + if (null == fsPermission) { + return null; + } + FsAction userAction = fsPermission.getUserAction() != null ? FsAction.valueOf(fsPermission.getUserAction()) : null; + FsAction grpAction = fsPermission.getGroupAction() != null ? FsAction.valueOf(fsPermission.getGroupAction()) : null; + FsAction otherAction = + fsPermission.getOtherAction() != null ? FsAction.valueOf(fsPermission.getOtherAction()) : null; + boolean stickyBit = fsPermission.getStickyBit() != null ? fsPermission.getStickyBit() : false; + return new FsPermission(userAction, grpAction, otherAction, stickyBit); + } + + public static HoodieFSPermission fromFSPermission(FsPermission fsPermission) { + if (null == fsPermission) { + return null; + } + String userAction = fsPermission.getUserAction() != null ? fsPermission.getUserAction().name() : null; + String grpAction = fsPermission.getGroupAction() != null ? fsPermission.getGroupAction().name() : null; + String otherAction = fsPermission.getOtherAction() != null ? fsPermission.getOtherAction().name() : null; + return HoodieFSPermission.newBuilder().setUserAction(userAction).setGroupAction(grpAction) + .setOtherAction(otherAction).setStickyBit(fsPermission.getStickyBit()).build(); + } + + public static FileStatus toFileStatus(HoodieFileStatus fileStatus) { + if (null == fileStatus) { + return null; + } + + return new FileStatus(fileStatus.getLength(), fileStatus.getIsDir() == null ? false : fileStatus.getIsDir(), + fileStatus.getBlockReplication(), fileStatus.getBlockSize(), fileStatus.getModificationTime(), + fileStatus.getAccessTime(), toFSPermission(fileStatus.getPermission()), fileStatus.getOwner(), + fileStatus.getGroup(), toPath(fileStatus.getSymlink()), toPath(fileStatus.getPath())); + } + + public static HoodieFileStatus fromFileStatus(FileStatus fileStatus) { + if (null == fileStatus) { + return null; + } + + HoodieFileStatus fStatus = new HoodieFileStatus(); + try { + fStatus.setPath(fromPath(fileStatus.getPath())); + fStatus.setLength(fileStatus.getLen()); + fStatus.setIsDir(fileStatus.isDirectory()); + fStatus.setBlockReplication((int) fileStatus.getReplication()); + fStatus.setBlockSize(fileStatus.getBlockSize()); + fStatus.setModificationTime(fileStatus.getModificationTime()); + fStatus.setAccessTime(fileStatus.getModificationTime()); + fStatus.setSymlink(fileStatus.isSymlink() ? fromPath(fileStatus.getSymlink()) : null); + safeReadAndSetMetadata(fStatus, fileStatus); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + return fStatus; + } + + /** + * Used to safely handle FileStatus calls which might fail on some FileSystem implementation. + * (DeprecatedLocalFileSystem) + */ + private static void safeReadAndSetMetadata(HoodieFileStatus fStatus, FileStatus fileStatus) { + try { + fStatus.setOwner(fileStatus.getOwner()); + fStatus.setGroup(fileStatus.getGroup()); + fStatus.setPermission(fromFSPermission(fileStatus.getPermission())); + } catch (IllegalArgumentException ie) { + // Deprecated File System (testing) does not work well with this call + // skipping + } + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java new file mode 100644 index 0000000000000..daeac06d018c4 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.bootstrap.index; + +import org.apache.hudi.common.model.BootstrapFileMapping; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.table.HoodieTableMetaClient; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ReflectionUtils; + +/** + * Bootstrap Index Interface. + */ +public abstract class BootstrapIndex implements Serializable { + + protected static final long serialVersionUID = 1L; + + protected final HoodieTableMetaClient metaClient; + + public BootstrapIndex(HoodieTableMetaClient metaClient) { + this.metaClient = metaClient; + } + + /** + * Create Bootstrap Index Reader. + * @return Index Reader + */ + public abstract IndexReader createReader(); + + /** + * Create Bootstrap Index Writer. + * @param sourceBasePath Source Base Path + * @return Index Writer + */ + public abstract IndexWriter createWriter(String sourceBasePath); + + /** + * Drop bootstrap index. + */ + public abstract void dropIndex(); + + /** + * Returns true if valid metadata bootstrap is present. + * @return + */ + public final boolean useIndex() { + boolean validInstantTime = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant() + .map(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, + HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)).orElse(false); + return validInstantTime && metaClient.getTableConfig().getBootstrapBasePath().isPresent() && isPresent(); + } + + /** + * Check if bootstrap Index is present and ensures readable. + */ + protected abstract boolean isPresent(); + + /** + * Bootstrap Index Reader Interface. + */ + public abstract static class IndexReader implements Serializable, AutoCloseable { + + protected final HoodieTableMetaClient metaClient; + + public IndexReader(HoodieTableMetaClient metaClient) { + this.metaClient = metaClient; + } + + /** + * Return Source base path. + * @return + */ + public abstract String getBootstrapBasePath(); + + /** + * Return list of partitions indexed. + * @return + */ + public abstract List getIndexedPartitionPaths(); + + /** + * Return list file-ids indexed. + * @return + */ + public abstract List getIndexedFileIds(); + + /** + * Lookup bootstrap index by partition. + * @param partition Partition to lookup + * @return + */ + public abstract List getSourceFileMappingForPartition(String partition); + + /** + * Lookup Bootstrap index by file group ids. + * @param ids File Group Ids + * @return + */ + public abstract Map getSourceFileMappingForFileIds( + List ids); + + public abstract void close(); + } + + /** + * Bootstrap Index Writer Interface. + */ + public abstract static class IndexWriter implements AutoCloseable { + + protected final HoodieTableMetaClient metaClient; + + public IndexWriter(HoodieTableMetaClient metaClient) { + this.metaClient = metaClient; + } + + /** + * Writer calls this method before beginning indexing partitions. + */ + public abstract void begin(); + + /** + * Append bootstrap index entries for next partitions in sorted order. + * @param partitionPath Partition Path + * @param bootstrapFileMappings Bootstrap Source File to File Id mapping + */ + public abstract void appendNextPartition(String partitionPath, + List bootstrapFileMappings); + + /** + * Writer calls this method after appending all partitions to be indexed. + */ + public abstract void finish(); + + public abstract void close(); + } + + public static BootstrapIndex getBootstrapIndex(HoodieTableMetaClient metaClient) { + return ((BootstrapIndex)(ReflectionUtils.loadClass( + metaClient.getTableConfig().getBootstrapIndexClass(), metaClient))); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java new file mode 100644 index 0000000000000..b9f745442b44d --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java @@ -0,0 +1,534 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.bootstrap.index; + +import org.apache.hudi.avro.model.HoodieBootstrapFilePartitionInfo; +import org.apache.hudi.avro.model.HoodieBootstrapIndexInfo; +import org.apache.hudi.avro.model.HoodieBootstrapPartitionMetadata; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.BootstrapFileMapping; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.io.hfile.HFileScanner; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Maintains mapping from skeleton file id to external bootstrap file. + * It maintains 2 physical indices. + * (a) At partition granularity to lookup all indices for each partition. + * (b) At file-group granularity to lookup bootstrap mapping for an individual file-group. + * + * This implementation uses HFile as physical storage of index. FOr the initial run, bootstrap + * mapping for the entire dataset resides in a single file but care has been taken in naming + * the index files in the same way as Hudi data files so that we can reuse file-system abstraction + * on these index files to manage multiple file-groups. + */ + +public class HFileBootstrapIndex extends BootstrapIndex { + + protected static final long serialVersionUID = 1L; + + private static final Logger LOG = LogManager.getLogger(HFileBootstrapIndex.class); + + public static final String BOOTSTRAP_INDEX_FILE_ID = "00000000-0000-0000-0000-000000000000-0"; + + // Additional Metadata written to HFiles. + public static final byte[] INDEX_INFO_KEY = Bytes.toBytes("INDEX_INFO"); + + private final boolean isPresent; + + public HFileBootstrapIndex(HoodieTableMetaClient metaClient) { + super(metaClient); + Path indexByPartitionPath = partitionIndexPath(metaClient); + Path indexByFilePath = fileIdIndexPath(metaClient); + try { + FileSystem fs = metaClient.getFs(); + isPresent = fs.exists(indexByPartitionPath) && fs.exists(indexByFilePath); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + private static String getPartitionKey(String partition) { + return "part=" + partition; + } + + private static String getFileGroupKey(HoodieFileGroupId fileGroupId) { + return "part=" + fileGroupId.getPartitionPath() + ";fileid=" + fileGroupId.getFileId(); + } + + private static Path partitionIndexPath(HoodieTableMetaClient metaClient) { + return new Path(metaClient.getBootstrapIndexByPartitionFolderPath(), + FSUtils.makeBootstrapIndexFileName(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, BOOTSTRAP_INDEX_FILE_ID, + HoodieFileFormat.HFILE.getFileExtension())); + } + + private static Path fileIdIndexPath(HoodieTableMetaClient metaClient) { + return new Path(metaClient.getBootstrapIndexByFileIdFolderNameFolderPath(), + FSUtils.makeBootstrapIndexFileName(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, BOOTSTRAP_INDEX_FILE_ID, + HoodieFileFormat.HFILE.getFileExtension())); + } + + /** + * Helper method to create HFile Reader. + * + * @param hFilePath File Path + * @param conf Configuration + * @param fileSystem File System + */ + private static HFile.Reader createReader(String hFilePath, Configuration conf, FileSystem fileSystem) { + try { + LOG.info("Opening HFile for reading :" + hFilePath); + HFile.Reader reader = HFile.createReader(fileSystem, new HFilePathForReader(hFilePath), + new CacheConfig(conf), conf); + return reader; + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + @Override + public BootstrapIndex.IndexReader createReader() { + return new HFileBootstrapIndexReader(metaClient); + } + + @Override + public BootstrapIndex.IndexWriter createWriter(String bootstrapBasePath) { + return new HFileBootstrapIndexWriter(bootstrapBasePath, metaClient); + } + + @Override + public void dropIndex() { + try { + Path[] indexPaths = new Path[]{partitionIndexPath(metaClient), fileIdIndexPath(metaClient)}; + for (Path indexPath : indexPaths) { + if (metaClient.getFs().exists(indexPath)) { + LOG.info("Dropping bootstrap index. Deleting file : " + indexPath); + metaClient.getFs().delete(indexPath); + } + } + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + @Override + protected boolean isPresent() { + return isPresent; + } + + /** + * HFile Based Index Reader. + */ + public static class HFileBootstrapIndexReader extends BootstrapIndex.IndexReader { + + // Base Path of external files. + private final String bootstrapBasePath; + // Well Known Paths for indices + private final String indexByPartitionPath; + private final String indexByFileIdPath; + + // Index Readers + private transient HFile.Reader indexByPartitionReader; + private transient HFile.Reader indexByFileIdReader; + + // Bootstrap Index Info + private transient HoodieBootstrapIndexInfo bootstrapIndexInfo; + + public HFileBootstrapIndexReader(HoodieTableMetaClient metaClient) { + super(metaClient); + Path indexByPartitionPath = partitionIndexPath(metaClient); + Path indexByFilePath = fileIdIndexPath(metaClient); + this.indexByPartitionPath = indexByPartitionPath.toString(); + this.indexByFileIdPath = indexByFilePath.toString(); + initIndexInfo(); + this.bootstrapBasePath = bootstrapIndexInfo.getBootstrapBasePath(); + LOG.info("Loaded HFileBasedBootstrapIndex with source base path :" + bootstrapBasePath); + } + + private void initIndexInfo() { + synchronized (this) { + if (null == bootstrapIndexInfo) { + try { + bootstrapIndexInfo = fetchBootstrapIndexInfo(); + } catch (IOException ioe) { + throw new HoodieException(ioe.getMessage(), ioe); + } + } + } + } + + private HoodieBootstrapIndexInfo fetchBootstrapIndexInfo() throws IOException { + return TimelineMetadataUtils.deserializeAvroMetadata( + partitionIndexReader().loadFileInfo().get(INDEX_INFO_KEY), + HoodieBootstrapIndexInfo.class); + } + + private HFile.Reader partitionIndexReader() { + if (null == indexByPartitionReader) { + synchronized (this) { + if (null == indexByPartitionReader) { + LOG.info("Opening partition index :" + indexByPartitionPath); + this.indexByPartitionReader = + createReader(indexByPartitionPath, metaClient.getHadoopConf(), metaClient.getFs()); + } + } + } + return indexByPartitionReader; + } + + private HFile.Reader fileIdIndexReader() { + if (null == indexByFileIdReader) { + synchronized (this) { + if (null == indexByFileIdReader) { + LOG.info("Opening fileId index :" + indexByFileIdPath); + this.indexByFileIdReader = + createReader(indexByFileIdPath, metaClient.getHadoopConf(), metaClient.getFs()); + } + } + } + return indexByFileIdReader; + } + + @Override + public List getIndexedPartitionPaths() { + HFileScanner scanner = partitionIndexReader().getScanner(true, true); + return getAllKeys(scanner); + } + + @Override + public List getIndexedFileIds() { + HFileScanner scanner = fileIdIndexReader().getScanner(true, true); + return getAllKeys(scanner); + } + + private List getAllKeys(HFileScanner scanner) { + List keys = new ArrayList<>(); + try { + boolean available = scanner.seekTo(); + while (available) { + keys.add(CellUtil.getCellKeyAsString(scanner.getKeyValue())); + available = scanner.next(); + } + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + + return keys; + } + + @Override + public List getSourceFileMappingForPartition(String partition) { + try { + HFileScanner scanner = partitionIndexReader().getScanner(true, true); + KeyValue keyValue = new KeyValue(Bytes.toBytes(getPartitionKey(partition)), new byte[0], new byte[0], + HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put, new byte[0]); + if (scanner.seekTo(keyValue) == 0) { + ByteBuffer readValue = scanner.getValue(); + byte[] valBytes = Bytes.toBytes(readValue); + HoodieBootstrapPartitionMetadata metadata = + TimelineMetadataUtils.deserializeAvroMetadata(valBytes, HoodieBootstrapPartitionMetadata.class); + return metadata.getFileIdToBootstrapFile().entrySet().stream() + .map(e -> new BootstrapFileMapping(bootstrapBasePath, metadata.getBootstrapPartitionPath(), + partition, e.getValue(), e.getKey())).collect(Collectors.toList()); + } else { + LOG.warn("No value found for partition key (" + partition + ")"); + return new ArrayList<>(); + } + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + @Override + public String getBootstrapBasePath() { + return bootstrapBasePath; + } + + @Override + public Map getSourceFileMappingForFileIds( + List ids) { + Map result = new HashMap<>(); + // Arrange input Keys in sorted order for 1 pass scan + List fileGroupIds = new ArrayList<>(ids); + Collections.sort(fileGroupIds); + try { + HFileScanner scanner = fileIdIndexReader().getScanner(true, true); + for (HoodieFileGroupId fileGroupId : fileGroupIds) { + KeyValue keyValue = new KeyValue(Bytes.toBytes(getFileGroupKey(fileGroupId)), new byte[0], new byte[0], + HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put, new byte[0]); + if (scanner.seekTo(keyValue) == 0) { + ByteBuffer readValue = scanner.getValue(); + byte[] valBytes = Bytes.toBytes(readValue); + HoodieBootstrapFilePartitionInfo fileInfo = TimelineMetadataUtils.deserializeAvroMetadata(valBytes, + HoodieBootstrapFilePartitionInfo.class); + BootstrapFileMapping mapping = new BootstrapFileMapping(bootstrapBasePath, + fileInfo.getBootstrapPartitionPath(), fileInfo.getPartitionPath(), fileInfo.getBootstrapFileStatus(), + fileGroupId.getFileId()); + result.put(fileGroupId, mapping); + } + } + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + return result; + } + + @Override + public void close() { + try { + if (indexByPartitionReader != null) { + indexByPartitionReader.close(true); + indexByPartitionReader = null; + } + if (indexByFileIdReader != null) { + indexByFileIdReader.close(true); + indexByFileIdReader = null; + } + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + } + + /** + * Boostrap Index Writer to build bootstrap index. + */ + public static class HFileBootstrapIndexWriter extends BootstrapIndex.IndexWriter { + + private final String bootstrapBasePath; + private final Path indexByPartitionPath; + private final Path indexByFileIdPath; + private HFile.Writer indexByPartitionWriter; + private HFile.Writer indexByFileIdWriter; + + private boolean closed = false; + private int numPartitionKeysAdded = 0; + private int numFileIdKeysAdded = 0; + + private final Map> sourceFileMappings = new HashMap<>(); + + private HFileBootstrapIndexWriter(String bootstrapBasePath, HoodieTableMetaClient metaClient) { + super(metaClient); + try { + metaClient.initializeBootstrapDirsIfNotExists(); + this.bootstrapBasePath = bootstrapBasePath; + this.indexByPartitionPath = partitionIndexPath(metaClient); + this.indexByFileIdPath = fileIdIndexPath(metaClient); + + if (metaClient.getFs().exists(indexByPartitionPath) || metaClient.getFs().exists(indexByFileIdPath)) { + String errMsg = "Previous version of bootstrap index exists. Partition Index Path :" + indexByPartitionPath + + ", FileId index Path :" + indexByFileIdPath; + LOG.info(errMsg); + throw new HoodieException(errMsg); + } + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + /** + * Append bootstrap index entries for next partitions in sorted order. + * @param partitionPath Hudi Partition Path + * @param bootstrapPartitionPath Source Partition Path + * @param bootstrapFileMappings Bootstrap Source File to Hudi File Id mapping + */ + private void writeNextPartition(String partitionPath, String bootstrapPartitionPath, + List bootstrapFileMappings) { + try { + LOG.info("Adding bootstrap partition Index entry for partition :" + partitionPath + + ", bootstrap Partition :" + bootstrapPartitionPath + ", Num Entries :" + bootstrapFileMappings.size()); + LOG.info("ADDING entries :" + bootstrapFileMappings); + HoodieBootstrapPartitionMetadata bootstrapPartitionMetadata = new HoodieBootstrapPartitionMetadata(); + bootstrapPartitionMetadata.setBootstrapPartitionPath(bootstrapPartitionPath); + bootstrapPartitionMetadata.setPartitionPath(partitionPath); + bootstrapPartitionMetadata.setFileIdToBootstrapFile( + bootstrapFileMappings.stream().map(m -> Pair.of(m.getFileId(), + m.getBoostrapFileStatus())).collect(Collectors.toMap(Pair::getKey, Pair::getValue))); + Option bytes = TimelineMetadataUtils.serializeAvroMetadata(bootstrapPartitionMetadata, HoodieBootstrapPartitionMetadata.class); + if (bytes.isPresent()) { + indexByPartitionWriter + .append(new KeyValue(Bytes.toBytes(getPartitionKey(partitionPath)), new byte[0], new byte[0], + HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put, bytes.get())); + numPartitionKeysAdded++; + } + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } + + /** + * Write next source file to hudi file-id. Entries are expected to be appended in hudi file-group id + * order. + * @param mapping boostrap source file mapping. + */ + private void writeNextSourceFileMapping(BootstrapFileMapping mapping) { + try { + HoodieBootstrapFilePartitionInfo srcFilePartitionInfo = new HoodieBootstrapFilePartitionInfo(); + srcFilePartitionInfo.setPartitionPath(mapping.getPartitionPath()); + srcFilePartitionInfo.setBootstrapPartitionPath(mapping.getBootstrapPartitionPath()); + srcFilePartitionInfo.setBootstrapFileStatus(mapping.getBoostrapFileStatus()); + KeyValue kv = new KeyValue(getFileGroupKey(mapping.getFileGroupId()).getBytes(), new byte[0], new byte[0], + HConstants.LATEST_TIMESTAMP, KeyValue.Type.Put, + TimelineMetadataUtils.serializeAvroMetadata(srcFilePartitionInfo, + HoodieBootstrapFilePartitionInfo.class).get()); + indexByFileIdWriter.append(kv); + numFileIdKeysAdded++; + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } + + /** + * Commit bootstrap index entries. Appends Metadata and closes write handles. + */ + private void commit() { + try { + if (!closed) { + HoodieBootstrapIndexInfo partitionIndexInfo = HoodieBootstrapIndexInfo.newBuilder() + .setCreatedTimestamp(new Date().getTime()) + .setNumKeys(numPartitionKeysAdded) + .setBootstrapBasePath(bootstrapBasePath) + .build(); + LOG.info("Adding Partition FileInfo :" + partitionIndexInfo); + + HoodieBootstrapIndexInfo fileIdIndexInfo = HoodieBootstrapIndexInfo.newBuilder() + .setCreatedTimestamp(new Date().getTime()) + .setNumKeys(numFileIdKeysAdded) + .setBootstrapBasePath(bootstrapBasePath) + .build(); + LOG.info("Appending FileId FileInfo :" + fileIdIndexInfo); + + indexByPartitionWriter.appendFileInfo(INDEX_INFO_KEY, + TimelineMetadataUtils.serializeAvroMetadata(partitionIndexInfo, HoodieBootstrapIndexInfo.class).get()); + indexByFileIdWriter.appendFileInfo(INDEX_INFO_KEY, + TimelineMetadataUtils.serializeAvroMetadata(fileIdIndexInfo, HoodieBootstrapIndexInfo.class).get()); + + close(); + } + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + /** + * Close Writer Handles. + */ + public void close() { + try { + if (!closed) { + indexByPartitionWriter.close(); + indexByFileIdWriter.close(); + closed = true; + } + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + @Override + public void begin() { + try { + HFileContext meta = new HFileContextBuilder().build(); + this.indexByPartitionWriter = HFile.getWriterFactory(metaClient.getHadoopConf(), + new CacheConfig(metaClient.getHadoopConf())).withPath(metaClient.getFs(), indexByPartitionPath) + .withFileContext(meta).withComparator(new HoodieKVComparator()).create(); + this.indexByFileIdWriter = HFile.getWriterFactory(metaClient.getHadoopConf(), + new CacheConfig(metaClient.getHadoopConf())).withPath(metaClient.getFs(), indexByFileIdPath) + .withFileContext(meta).withComparator(new HoodieKVComparator()).create(); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + @Override + public void appendNextPartition(String partitionPath, List bootstrapFileMappings) { + sourceFileMappings.put(partitionPath, bootstrapFileMappings); + } + + @Override + public void finish() { + // Sort and write + List partitions = sourceFileMappings.keySet().stream().sorted().collect(Collectors.toList()); + partitions.forEach(p -> writeNextPartition(p, sourceFileMappings.get(p).get(0).getBootstrapPartitionPath(), + sourceFileMappings.get(p))); + sourceFileMappings.values().stream().flatMap(Collection::stream).sorted() + .forEach(this::writeNextSourceFileMapping); + commit(); + } + } + + /** + * IMPORTANT : + * HFile Readers use HFile name (instead of path) as cache key. This could be fine as long + * as file names are UUIDs. For bootstrap, we are using well-known index names. + * Hence, this hacky workaround to return full path string from Path subclass and pass it to reader. + * The other option is to disable block cache for Bootstrap which again involves some custom code + * as there is no API to disable cache. + */ + private static class HFilePathForReader extends Path { + + public HFilePathForReader(String pathString) throws IllegalArgumentException { + super(pathString); + } + + @Override + public String getName() { + return toString(); + } + } + + /** + * This class is explicitly used as Key Comparator to workaround hard coded + * legacy format class names inside HBase. Otherwise we will face issues with shading. + */ + public static class HoodieKVComparator extends KeyValue.KVComparator { + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index 6807f6f5d6eaf..542c77931f524 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.fs; +import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; @@ -45,6 +46,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; + import java.util.List; import java.util.Map.Entry; import java.util.Objects; @@ -52,6 +54,7 @@ import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -114,6 +117,10 @@ public static String makeDataFileName(String instantTime, String writeToken, Str return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, fileExtension); } + public static String makeBootstrapIndexFileName(String instantTime, String fileId, String ext) { + return String.format("%s_%s_%s%s", fileId, "1-0-1", instantTime, ext); + } + public static String maskWithoutFileId(String instantTime, int taskPartitionId) { return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieFileFormat.PARQUET.getFileExtension()); } @@ -225,8 +232,8 @@ public static List getAllPartitionPaths(FileSystem fs, String basePathSt public static String getFileExtension(String fullName) { Objects.requireNonNull(fullName); - String fileName = (new File(fullName)).getName(); - int dotIndex = fileName.indexOf('.'); + String fileName = new File(fullName).getName(); + int dotIndex = fileName.lastIndexOf('.'); return dotIndex == -1 ? "" : fileName.substring(dotIndex); } @@ -235,10 +242,6 @@ private static PathFilter getExcludeMetaPathFilter() { return (path) -> !path.toString().contains(HoodieTableMetaClient.METAFOLDER_NAME); } - public static String getInstantTime(String name) { - return name.replace(getFileExtension(name), ""); - } - /** * Returns a new unique prefix for creating a file group. */ @@ -516,4 +519,33 @@ public static Configuration registerFileSystem(Path file, Configuration conf) { return returnConf; } + /** + * Get the FS implementation for this table. + * @param path Path String + * @param hadoopConf Serializable Hadoop Configuration + * @param consistencyGuardConfig Consistency Guard Config + * @return HoodieWrapperFileSystem + */ + public static HoodieWrapperFileSystem getFs(String path, SerializableConfiguration hadoopConf, + ConsistencyGuardConfig consistencyGuardConfig) { + FileSystem fileSystem = FSUtils.getFs(path, hadoopConf.newCopy()); + return new HoodieWrapperFileSystem(fileSystem, + consistencyGuardConfig.isConsistencyCheckEnabled() + ? new FailSafeConsistencyGuard(fileSystem, consistencyGuardConfig) + : new NoOpConsistencyGuard()); + } + + /** + * Helper to filter out paths under metadata folder when running fs.globStatus. + * @param fs File System + * @param globPath Glob Path + * @return + * @throws IOException + */ + public static List getGlobStatusExcludingMetaFolder(FileSystem fs, Path globPath) throws IOException { + FileStatus[] statuses = fs.globStatus(globPath); + return Arrays.stream(statuses) + .filter(fileStatus -> !fileStatus.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME)) + .collect(Collectors.toList()); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/BaseFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/BaseFile.java new file mode 100644 index 0000000000000..f12c207ee75b6 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/BaseFile.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import java.io.Serializable; +import java.util.Objects; + +/** + * Represents common metadata about base-file. + * A base file can be Hudi base file or even an external (non-hudi) base file too. + */ +public class BaseFile implements Serializable { + + private static final long serialVersionUID = 1L; + private transient FileStatus fileStatus; + private final String fullPath; + private long fileLen; + + public BaseFile(BaseFile dataFile) { + this.fileStatus = dataFile.fileStatus; + this.fullPath = dataFile.fullPath; + this.fileLen = dataFile.fileLen; + } + + public BaseFile(FileStatus fileStatus) { + this.fileStatus = fileStatus; + this.fullPath = fileStatus.getPath().toString(); + this.fileLen = fileStatus.getLen(); + } + + public BaseFile(String filePath) { + this.fileStatus = null; + this.fullPath = filePath; + this.fileLen = -1; + } + + public String getPath() { + return fullPath; + } + + public String getFileName() { + return new Path(fullPath).getName(); + } + + public FileStatus getFileStatus() { + return fileStatus; + } + + public long getFileSize() { + return fileLen; + } + + public void setFileLen(long fileLen) { + this.fileLen = fileLen; + } + + public long getFileLen() { + return fileLen; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BaseFile dataFile = (BaseFile) o; + return Objects.equals(fullPath, dataFile.fullPath); + } + + @Override + public int hashCode() { + return Objects.hash(fullPath); + } + + @Override + public String toString() { + return "BaseFile{fullPath=" + fullPath + ", fileLen=" + fileLen + '}'; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/BootstrapBaseFileMapping.java b/hudi-common/src/main/java/org/apache/hudi/common/model/BootstrapBaseFileMapping.java new file mode 100644 index 0000000000000..af375038c5104 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/BootstrapBaseFileMapping.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import java.io.Serializable; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.common.bootstrap.FileStatusUtils; + +/** + * POJO storing (partitionPath, hoodieFileId) -> external base file path. + */ +public class BootstrapBaseFileMapping implements Serializable { + + private final HoodieFileGroupId fileGroupId; + + private final HoodieFileStatus bootstrapFileStatus; + + public BootstrapBaseFileMapping(HoodieFileGroupId fileGroupId, HoodieFileStatus bootstrapFileStatus) { + this.fileGroupId = fileGroupId; + this.bootstrapFileStatus = bootstrapFileStatus; + } + + public HoodieFileGroupId getFileGroupId() { + return fileGroupId; + } + + public BaseFile getBootstrapBaseFile() { + return new BaseFile(FileStatusUtils.toFileStatus(bootstrapFileStatus)); + } + + @Override + public String toString() { + return "BootstrapBaseFileMapping{" + + "fileGroupId=" + fileGroupId + + ", bootstrapFileStatus=" + bootstrapFileStatus + + '}'; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/BootstrapFileMapping.java b/hudi-common/src/main/java/org/apache/hudi/common/model/BootstrapFileMapping.java new file mode 100644 index 0000000000000..a9642c71b3ac2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/BootstrapFileMapping.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import java.io.Serializable; +import java.util.Objects; +import org.apache.hudi.avro.model.HoodieFileStatus; + +/** + * Value stored in the bootstrap index. + */ +public class BootstrapFileMapping implements Serializable, Comparable { + + private final String bootstrapBasePath; + private final String bootstrapPartitionPath; + private final HoodieFileStatus boostrapFileStatus; + + private final String partitionPath; + private final String fileId; + + public BootstrapFileMapping(String bootstrapBasePath, String bootstrapPartitionPath, String partitionPath, + HoodieFileStatus boostrapFileStatus, String fileId) { + this.bootstrapBasePath = bootstrapBasePath; + this.bootstrapPartitionPath = bootstrapPartitionPath; + this.partitionPath = partitionPath; + this.boostrapFileStatus = boostrapFileStatus; + this.fileId = fileId; + } + + @Override + public String toString() { + return "BootstrapFileMapping{" + + "bootstrapBasePath='" + bootstrapBasePath + '\'' + + ", bootstrapPartitionPath='" + bootstrapPartitionPath + '\'' + + ", boostrapFileStatus=" + boostrapFileStatus + + ", partitionPath='" + partitionPath + '\'' + + ", fileId='" + fileId + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BootstrapFileMapping mapping = (BootstrapFileMapping) o; + return Objects.equals(bootstrapBasePath, mapping.bootstrapBasePath) + && Objects.equals(bootstrapPartitionPath, mapping.bootstrapPartitionPath) + && Objects.equals(partitionPath, mapping.partitionPath) + && Objects.equals(boostrapFileStatus, mapping.boostrapFileStatus) + && Objects.equals(fileId, mapping.fileId); + } + + @Override + public int hashCode() { + return Objects.hash(bootstrapBasePath, bootstrapPartitionPath, partitionPath, boostrapFileStatus, fileId); + } + + public String getBootstrapBasePath() { + return bootstrapBasePath; + } + + public String getBootstrapPartitionPath() { + return bootstrapPartitionPath; + } + + public String getPartitionPath() { + return partitionPath; + } + + public HoodieFileStatus getBoostrapFileStatus() { + return boostrapFileStatus; + } + + public String getFileId() { + return fileId; + } + + public HoodieFileGroupId getFileGroupId() { + return new HoodieFileGroupId(partitionPath, fileId); + } + + @Override + public int compareTo(BootstrapFileMapping o) { + int ret = partitionPath.compareTo(o.partitionPath); + if (ret == 0) { + ret = fileId.compareTo(o.fileId); + } + return ret; + } +} 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 664001fed82cc..14f8f59b3daea 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 @@ -44,18 +44,20 @@ public class CompactionOperation implements Serializable { private Option dataFileName; private HoodieFileGroupId id; private Map metrics; + private Option bootstrapFilePath; // Only for serialization/de-serialization @Deprecated public CompactionOperation() {} public CompactionOperation(String fileId, String partitionPath, String baseInstantTime, - Option dataFileCommitTime, List deltaFileNames, Option dataFileName, - Map metrics) { + Option dataFileCommitTime, List deltaFileNames, Option dataFileName, + Option bootstrapFilePath, Map metrics) { this.baseInstantTime = baseInstantTime; this.dataFileCommitTime = dataFileCommitTime; this.deltaFileNames = deltaFileNames; this.dataFileName = dataFileName; + this.bootstrapFilePath = bootstrapFilePath; this.id = new HoodieFileGroupId(partitionPath, fileId); this.metrics = metrics; } @@ -67,14 +69,15 @@ public CompactionOperation(Option dataFile, String partitionPath this.dataFileName = Option.of(dataFile.get().getFileName()); this.id = new HoodieFileGroupId(partitionPath, dataFile.get().getFileId()); this.dataFileCommitTime = Option.of(dataFile.get().getCommitTime()); + this.bootstrapFilePath = dataFile.get().getBootstrapBaseFile().map(BaseFile::getPath); } else { assert logFiles.size() > 0; this.dataFileName = Option.empty(); this.baseInstantTime = FSUtils.getBaseCommitTimeFromLogPath(logFiles.get(0).getPath()); this.id = new HoodieFileGroupId(partitionPath, FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath())); this.dataFileCommitTime = Option.empty(); + this.bootstrapFilePath = Option.empty(); } - this.deltaFileNames = logFiles.stream().map(s -> s.getPath().getName()).collect(Collectors.toList()); this.metrics = metrics; } @@ -111,9 +114,17 @@ public HoodieFileGroupId getFileGroupId() { return id; } + public Option getBootstrapFilePath() { + return bootstrapFilePath; + } + public Option getBaseFile(String basePath, String partitionPath) { + Option externalBaseFile = bootstrapFilePath.map(BaseFile::new); Path dirPath = FSUtils.getPartitionPath(basePath, partitionPath); - return dataFileName.map(df -> new HoodieBaseFile(new Path(dirPath, df).toString())); + return dataFileName.map(df -> { + return externalBaseFile.map(ext -> new HoodieBaseFile(new Path(dirPath, df).toString(), ext)) + .orElseGet(() -> new HoodieBaseFile(new Path(dirPath, df).toString())); + }); } /** @@ -130,6 +141,7 @@ public static CompactionOperation convertFromAvroRecordInstance(HoodieCompaction op.deltaFileNames = new ArrayList<>(operation.getDeltaFilePaths()); op.id = new HoodieFileGroupId(operation.getPartitionPath(), operation.getFileId()); op.metrics = operation.getMetrics() == null ? new HashMap<>() : new HashMap<>(operation.getMetrics()); + op.bootstrapFilePath = Option.ofNullable(operation.getBootstrapFilePath()); return op; } @@ -137,7 +149,7 @@ public static CompactionOperation convertFromAvroRecordInstance(HoodieCompaction public String toString() { return "CompactionOperation{baseInstantTime='" + baseInstantTime + '\'' + ", dataFileCommitTime=" + dataFileCommitTime + ", deltaFileNames=" + deltaFileNames + ", dataFileName=" + dataFileName + ", id='" + id - + '\'' + ", metrics=" + metrics + '}'; + + '\'' + ", metrics=" + metrics + ", bootstrapFilePath=" + bootstrapFilePath + '}'; } @Override 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 19e62f999bac2..688e72bd786a2 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 @@ -52,6 +52,14 @@ public class FileSlice implements Serializable { */ private final TreeSet logFiles; + public FileSlice(FileSlice fileSlice) { + this.baseInstantTime = fileSlice.baseInstantTime; + this.baseFile = fileSlice.baseFile != null ? new HoodieBaseFile(fileSlice.baseFile) : null; + this.fileGroupId = fileSlice.fileGroupId; + this.logFiles = new TreeSet<>(HoodieLogFile.getReverseLogFileComparator()); + fileSlice.logFiles.forEach(lf -> this.logFiles.add(new HoodieLogFile(lf))); + } + public FileSlice(String partitionPath, String baseInstantTime, String fileId) { this(new HoodieFileGroupId(partitionPath, fileId), baseInstantTime); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java index 5ccc2a8728681..2c640bb143294 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java @@ -19,86 +19,60 @@ package org.apache.hudi.common.model; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.util.Option; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; - -import java.io.Serializable; -import java.util.Objects; /** - * Hoodie base file. + * Hoodie base file - Represents metadata about Hudi file in DFS. + * Supports APIs to get Hudi FileId, Commit Time and bootstrap file (if any). */ -public class HoodieBaseFile implements Serializable { - - private static final long serialVersionUID = 1L; - private transient FileStatus fileStatus; - private final String fullPath; - private long fileLen; +public class HoodieBaseFile extends BaseFile { - public HoodieBaseFile(FileStatus fileStatus) { - this.fileStatus = fileStatus; - this.fullPath = fileStatus.getPath().toString(); - this.fileLen = fileStatus.getLen(); - } + private Option bootstrapBaseFile; - public HoodieBaseFile(String filePath) { - this.fileStatus = null; - this.fullPath = filePath; - this.fileLen = -1; + public HoodieBaseFile(HoodieBaseFile dataFile) { + super(dataFile); + this.bootstrapBaseFile = dataFile.bootstrapBaseFile; } - public String getFileId() { - return FSUtils.getFileId(getFileName()); - } - - public String getCommitTime() { - return FSUtils.getCommitTime(getFileName()); - } - - public String getPath() { - return fullPath; + public HoodieBaseFile(FileStatus fileStatus) { + this(fileStatus, null); } - public String getFileName() { - return new Path(fullPath).getName(); + public HoodieBaseFile(FileStatus fileStatus, BaseFile bootstrapBaseFile) { + super(fileStatus); + this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile); } - public FileStatus getFileStatus() { - return fileStatus; + public HoodieBaseFile(String filePath) { + this(filePath, null); } - public long getFileSize() { - return fileLen; + public HoodieBaseFile(String filePath, BaseFile bootstrapBaseFile) { + super(filePath); + this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile); } - public void setFileLen(long fileLen) { - this.fileLen = fileLen; + public String getFileId() { + return FSUtils.getFileId(getFileName()); } - public long getFileLen() { - return fileLen; + public String getCommitTime() { + return FSUtils.getCommitTime(getFileName()); } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - HoodieBaseFile dataFile = (HoodieBaseFile) o; - return Objects.equals(fullPath, dataFile.fullPath); + public Option getBootstrapBaseFile() { + return bootstrapBaseFile; } - @Override - public int hashCode() { - return Objects.hash(fullPath); + public void setBootstrapBaseFile(BaseFile bootstrapBaseFile) { + this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile); } @Override public String toString() { - return "HoodieDataFile{fullPath=" + fullPath + ", fileLen=" + fileLen + '}'; + return "HoodieBaseFile{fullPath=" + getPath() + ", fileLen=" + getFileLen() + + ", BootstrapBaseFile=" + bootstrapBaseFile.orElse(null) + '}'; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileFormat.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileFormat.java index 43a9e5657314a..552c38ffd9bd1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileFormat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileFormat.java @@ -22,7 +22,9 @@ * Hoodie file format. */ public enum HoodieFileFormat { - PARQUET(".parquet"), HOODIE_LOG(".log"); + PARQUET(".parquet"), + HOODIE_LOG(".log"), + HFILE(".hfile"); private final String extension; 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 ba587e193e5e5..849f08eed697b 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 @@ -21,12 +21,14 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import java.io.Serializable; import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.TreeMap; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -58,6 +60,15 @@ public static Comparator getReverseCommitTimeComparator() { */ private final Option lastInstant; + public HoodieFileGroup(HoodieFileGroup fileGroup) { + this.timeline = fileGroup.timeline; + this.fileGroupId = fileGroup.fileGroupId; + this.fileSlices = new TreeMap<>(fileGroup.fileSlices.entrySet().stream() + .map(e -> Pair.of(e.getKey(), new FileSlice(e.getValue()))) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue))); + this.lastInstant = fileGroup.lastInstant; + } + public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) { this(new HoodieFileGroupId(partitionPath, id), timeline); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroupId.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroupId.java index eb0fbd5806138..ceea16af7aec3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroupId.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroupId.java @@ -24,7 +24,7 @@ /** * Unique ID to identify a file-group in a data-set. */ -public class HoodieFileGroupId implements Serializable { +public class HoodieFileGroupId implements Serializable, Comparable { private final String partitionPath; @@ -64,4 +64,13 @@ public int hashCode() { public String toString() { return "HoodieFileGroupId{partitionPath='" + partitionPath + '\'' + ", fileId='" + fileId + '\'' + '}'; } + + @Override + public int compareTo(HoodieFileGroupId o) { + int ret = partitionPath.compareTo(o.partitionPath); + if (ret == 0) { + ret = fileId.compareTo(fileId); + } + return ret; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java index 973fd519a4c19..fa7f9b1bbe5d7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java @@ -45,6 +45,12 @@ public class HoodieLogFile implements Serializable { private final String pathStr; private long fileLen; + public HoodieLogFile(HoodieLogFile logFile) { + this.fileStatus = logFile.fileStatus; + this.pathStr = logFile.pathStr; + this.fileLen = logFile.fileLen; + } + public HoodieLogFile(FileStatus fileStatus) { this.fileStatus = fileStatus; this.pathStr = fileStatus.getPath().toString(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index 23ba7f79c908f..d1a53d5da2ad6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -37,6 +37,7 @@ public enum WriteOperationType { BULK_INSERT_PREPPED("bulk_insert_prepped"), // delete DELETE("delete"), + BOOTSTRAP("bootstrap"), // used for old version UNKNOWN("unknown"); 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 2742e69f0a952..f188f3a865ccd 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 @@ -18,6 +18,7 @@ package org.apache.hudi.common.table; +import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; @@ -63,11 +64,15 @@ public class HoodieTableConfig implements Serializable { 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 String HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME = "hoodie.bootstrap.index.class"; + public static final String HOODIE_BOOTSTRAP_BASE_PATH = "hoodie.bootstrap.base.path"; public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE; 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 String DEFAULT_BOOTSTRAP_INDEX_CLASS = HFileBootstrapIndex.class.getName(); + public static final Integer DEFAULT_TIMELINE_LAYOUT_VERSION = TimelineLayoutVersion.VERSION_0; public static final String DEFAULT_ARCHIVELOG_FOLDER = ""; private Properties props; @@ -131,6 +136,10 @@ public static void createHoodieProperties(FileSystem fs, Path metadataFolder, Pr // Use latest Version as default unless forced by client properties.setProperty(HOODIE_TIMELINE_LAYOUT_VERSION, TimelineLayoutVersion.CURR_VERSION.toString()); } + if (properties.containsKey(HOODIE_BOOTSTRAP_BASE_PATH) && !properties.containsKey(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME)) { + // Use the default bootstrap index class. + properties.setProperty(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, DEFAULT_BOOTSTRAP_INDEX_CLASS); + } properties.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); } } @@ -161,6 +170,19 @@ public String getPayloadClass() { "org.apache.hudi"); } + /** + * Read the payload class for HoodieRecords from the table properties. + */ + public String getBootstrapIndexClass() { + // There could be tables written with payload class from com.uber.hoodie. Need to transparently + // change to org.apache.hudi + return props.getProperty(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, DEFAULT_BOOTSTRAP_INDEX_CLASS); + } + + public Option getBootstrapBasePath() { + return Option.ofNullable(props.getProperty(HOODIE_BOOTSTRAP_BASE_PATH)); + } + /** * Read the table name. */ 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 b0475956a7ac5..a74728eb52375 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 @@ -74,6 +74,12 @@ public class HoodieTableMetaClient implements Serializable { public static final String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp"; public static final String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + File.separator + ".aux"; public static final String BOOTSTRAP_INDEX_ROOT_FOLDER_PATH = AUXILIARYFOLDER_NAME + File.separator + ".bootstrap"; + + public static final String BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + + File.separator + ".partitions"; + public static final String BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + File.separator + + ".fileids"; + public static final String MARKER_EXTN = ".marker"; private String basePath; @@ -209,6 +215,20 @@ public String getMetaAuxiliaryPath() { return basePath + File.separator + AUXILIARYFOLDER_NAME; } + /** + * @return Bootstrap Index By Partition Folder + */ + public String getBootstrapIndexByPartitionFolderPath() { + return basePath + File.separator + BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH; + } + + /** + * @return Bootstrap Index By Hudi File Id Folder + */ + public String getBootstrapIndexByFileIdFolderNameFolderPath() { + return basePath + File.separator + BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH; + } + /** * @return path where archived timeline is stored */ @@ -300,44 +320,45 @@ public synchronized HoodieArchivedTimeline getArchivedTimeline() { return archivedTimeline; } - /** - * Helper method to initialize a table, with given basePath, tableType, name, archiveFolder, payloadClass. - */ - public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType, - String tableName, String archiveLogFolder, String payloadClassName) throws IOException { - return initTableType(hadoopConf, basePath, HoodieTableType.valueOf(tableType), tableName, - archiveLogFolder, payloadClassName, null, null); - } - /** * Helper method to initialize a table, with given basePath, tableType, name, archiveFolder, payloadClass and * base file format. */ - public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType, - String tableName, String archiveLogFolder, String payloadClassName, String baseFileFormat) throws IOException { - return initTableType(hadoopConf, basePath, HoodieTableType.valueOf(tableType), tableName, - archiveLogFolder, payloadClassName, null, baseFileFormat); + public static HoodieTableMetaClient initTableTypeWithBootstrap(Configuration hadoopConf, String basePath, HoodieTableType tableType, + String tableName, String archiveLogFolder, String payloadClassName, + String baseFileFormat, String bootstrapIndexClass, + String bootstrapBasePath) throws IOException { + return initTableType(hadoopConf, basePath, tableType, tableName, + archiveLogFolder, payloadClassName, null, baseFileFormat, bootstrapIndexClass, bootstrapBasePath); + } + + public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType, + String tableName, String archiveLogFolder, String payloadClassName, + String baseFileFormat) throws IOException { + return initTableType(hadoopConf, basePath, tableType, tableName, + archiveLogFolder, payloadClassName, null, baseFileFormat, null, null); } /** - * Helper method to initialize a given path, as a given type and table name. + * Used primarily by tests, examples. */ - public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, - HoodieTableType tableType, String tableName, String payloadClassName) throws IOException { - return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName, null, null); + public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType, + String tableName, String payloadClassName) throws IOException { + return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName, + null, null, null, null); } - public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, - HoodieTableType tableType, String tableName, - String archiveLogFolder, String payloadClassName, + public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType, + String tableName, String archiveLogFolder, String payloadClassName, Integer timelineLayoutVersion) throws IOException { return initTableType(hadoopConf, basePath, tableType, tableName, archiveLogFolder, payloadClassName, - timelineLayoutVersion, null); + timelineLayoutVersion, null, null, null); } - public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, - HoodieTableType tableType, String tableName, String archiveLogFolder, String payloadClassName, - Integer timelineLayoutVersion, String baseFileFormat) throws IOException { + private static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType, + String tableName, String archiveLogFolder, String payloadClassName, + Integer timelineLayoutVersion, String baseFileFormat, + String bootstrapIndexClass, String bootstrapBasePath) throws IOException { Properties properties = new Properties(); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name()); @@ -356,6 +377,15 @@ public static HoodieTableMetaClient initTableType(Configuration hadoopConf, Stri if (null != baseFileFormat) { properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, baseFileFormat.toUpperCase()); } + + if (null != bootstrapIndexClass) { + properties.put(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, bootstrapIndexClass); + } + + if (null != bootstrapBasePath) { + properties.put(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH, bootstrapBasePath); + } + return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties); } @@ -399,6 +429,7 @@ public static HoodieTableMetaClient initTableAndGetMetaClient(Configuration hado fs.mkdirs(auxiliaryFolder); } + initializeBootstrapDirsIfNotExists(hadoopConf, basePath, fs); HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); // We should not use fs.getConf as this might be different from the original configuration // used to create the fs in unit tests @@ -407,6 +438,25 @@ public static HoodieTableMetaClient initTableAndGetMetaClient(Configuration hado return metaClient; } + public static void initializeBootstrapDirsIfNotExists(Configuration hadoopConf, + String basePath, FileSystem fs) throws IOException { + + // Create bootstrap index by partition folder if it does not exist + final Path bootstrap_index_folder_by_partition = + new Path(basePath, HoodieTableMetaClient.BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH); + if (!fs.exists(bootstrap_index_folder_by_partition)) { + fs.mkdirs(bootstrap_index_folder_by_partition); + } + + + // Create bootstrap index by partition folder if it does not exist + final Path bootstrap_index_folder_by_fileids = + new Path(basePath, HoodieTableMetaClient.BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH); + if (!fs.exists(bootstrap_index_folder_by_fileids)) { + fs.mkdirs(bootstrap_index_folder_by_fileids); + } + } + /** * Helper method to scan all hoodie-instant metafiles. * @@ -512,7 +562,7 @@ public List scanHoodieInstantsFromFileSystem(Path timelinePath, S HoodieTableMetaClient .scanFiles(getFs(), timelinePath, path -> { // Include only the meta files with extensions that needs to be included - String extension = FSUtils.getFileExtension(path.getName()); + String extension = HoodieInstant.getTimelineFileExtension(path.getName()); return includedExtensions.contains(extension); })).map(HoodieInstant::new); @@ -549,6 +599,10 @@ public String toString() { return sb.toString(); } + public void initializeBootstrapDirsIfNotExists() throws IOException { + initializeBootstrapDirsIfNotExists(getHadoopConf(), basePath, getFs()); + } + public void setBasePath(String basePath) { this.basePath = basePath; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java index aea414bc7e227..939ba8bbe0436 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java @@ -18,9 +18,7 @@ package org.apache.hudi.common.table.timeline; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.util.CollectionUtils; - import org.apache.hadoop.fs.FileStatus; import java.io.Serializable; @@ -53,6 +51,12 @@ public static String getComparableAction(String action) { return COMPARABLE_ACTIONS.getOrDefault(action, action); } + public static String getTimelineFileExtension(String fileName) { + Objects.requireNonNull(fileName); + int dotIndex = fileName.indexOf('.'); + return dotIndex == -1 ? "" : fileName.substring(dotIndex); + } + /** * Instant State. */ @@ -77,7 +81,7 @@ public enum State { public HoodieInstant(FileStatus fileStatus) { // First read the instant timestamp. [==>20170101193025<==].commit String fileName = fileStatus.getPath().getName(); - String fileExtension = FSUtils.getFileExtension(fileName); + String fileExtension = getTimelineFileExtension(fileName); timestamp = fileName.replace(fileExtension, ""); // Next read the action for this marker diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index 151c18bc28445..45b9e3404617f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -81,6 +81,13 @@ public interface HoodieTimeline extends Serializable { String INVALID_INSTANT_TS = "0"; + // Instant corresponding to pristine state of the table after its creation + String INIT_INSTANT_TS = "00000000000000"; + // Instant corresponding to METADATA bootstrapping of table/partitions + String METADATA_BOOTSTRAP_INSTANT_TS = "00000000000001"; + // Instant corresponding to full bootstrapping of table/partitions + String FULL_BOOTSTRAP_INSTANT_TS = "00000000000002"; + /** * Filter this timeline to just include the in-flights. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/BaseFileDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/BaseFileDTO.java index 408aafda9bfe4..3776c98355242 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/BaseFileDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/BaseFileDTO.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.table.timeline.dto; +import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.HoodieBaseFile; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; @@ -35,6 +36,8 @@ public class BaseFileDTO { private String fullPath; @JsonProperty("fileLen") private long fileLen; + @JsonProperty("bootstrapBaseFile") + private BaseFileDTO bootstrapBaseFile; public static HoodieBaseFile toHoodieBaseFile(BaseFileDTO dto) { if (null == dto) { @@ -48,18 +51,39 @@ public static HoodieBaseFile toHoodieBaseFile(BaseFileDTO dto) { baseFile = new HoodieBaseFile(dto.fullPath); baseFile.setFileLen(dto.fileLen); } + + baseFile.setBootstrapBaseFile(toBaseFile(dto.bootstrapBaseFile)); return baseFile; } - public static BaseFileDTO fromHoodieBaseFile(HoodieBaseFile dataFile) { - if (null == dataFile) { + private static BaseFile toBaseFile(BaseFileDTO dto) { + if (null == dto) { + return null; + } + + BaseFile baseFile; + if (null != dto.fileStatus) { + baseFile = new BaseFile(FileStatusDTO.toFileStatus(dto.fileStatus)); + } else { + baseFile = new BaseFile(dto.fullPath); + baseFile.setFileLen(dto.fileLen); + } + return baseFile; + } + + public static BaseFileDTO fromHoodieBaseFile(BaseFile baseFile) { + if (null == baseFile) { return null; } BaseFileDTO dto = new BaseFileDTO(); - dto.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus()); - dto.fullPath = dataFile.getPath(); - dto.fileLen = dataFile.getFileLen(); + dto.fileStatus = FileStatusDTO.fromFileStatus(baseFile.getFileStatus()); + dto.fullPath = baseFile.getPath(); + dto.fileLen = baseFile.getFileLen(); + if (baseFile instanceof HoodieBaseFile) { + dto.bootstrapBaseFile = ((HoodieBaseFile)baseFile).getBootstrapBaseFile() + .map(BaseFileDTO::fromHoodieBaseFile).orElse(null); + } return dto; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/CompactionOpDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/CompactionOpDTO.java index db19847bbb227..e001d94a28297 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/CompactionOpDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/CompactionOpDTO.java @@ -60,6 +60,9 @@ public class CompactionOpDTO { @JsonProperty("metrics") private Map metrics; + @JsonProperty("bootstrapBaseFile") + private String bootstrapBaseFile; + public static CompactionOpDTO fromCompactionOperation(String compactionInstantTime, CompactionOperation op) { CompactionOpDTO dto = new CompactionOpDTO(); dto.fileId = op.getFileId(); @@ -70,13 +73,14 @@ public static CompactionOpDTO fromCompactionOperation(String compactionInstantTi dto.deltaFilePaths = new ArrayList<>(op.getDeltaFileNames()); dto.partitionPath = op.getPartitionPath(); dto.metrics = op.getMetrics() == null ? new HashMap<>() : new HashMap<>(op.getMetrics()); + dto.bootstrapBaseFile = op.getBootstrapFilePath().orElse(null); return dto; } public static Pair toCompactionOperation(CompactionOpDTO dto) { return Pair.of(dto.compactionInstantTime, new CompactionOperation(dto.fileId, dto.partitionPath, dto.baseInstantTime, - Option.ofNullable(dto.dataFileCommitTime), dto.deltaFilePaths, Option.ofNullable(dto.dataFilePath), - dto.metrics)); + Option.ofNullable(dto.dataFileCommitTime), dto.deltaFilePaths, + Option.ofNullable(dto.dataFilePath), Option.ofNullable(dto.bootstrapBaseFile), dto.metrics)); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileStatusDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileStatusDTO.java index 508549f288893..5a1769e8e551d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileStatusDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileStatusDTO.java @@ -68,7 +68,7 @@ public static FileStatusDTO fromFileStatus(FileStatus fileStatus) { dto.blockReplication = fileStatus.getReplication(); dto.blocksize = fileStatus.getBlockSize(); dto.modificationTime = fileStatus.getModificationTime(); - dto.accessTime = fileStatus.getModificationTime(); + dto.accessTime = fileStatus.getAccessTime(); dto.symlink = fileStatus.isSymlink() ? FilePathDTO.fromPath(fileStatus.getSymlink()) : null; safeReadAndSetMetadata(dto, fileStatus); } catch (IOException ioe) { 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 4fdf20b1842e5..77fcd1614612f 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 @@ -18,8 +18,11 @@ package org.apache.hudi.common.table.view; +import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.BootstrapFileMapping; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.BootstrapBaseFileMapping; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; @@ -34,7 +37,6 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -56,6 +58,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS; + /** * Common thread-safe implementation for multiple TableFileSystemView Implementations. Provides uniform handling of (a) * Loading file-system views from underlying file-system (b) Pending compaction operations and changing file-system @@ -73,7 +77,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV private HoodieTimeline visibleCommitsAndCompactionTimeline; // Used to concurrently load and populate partition views - private ConcurrentHashMap addedPartitions = new ConcurrentHashMap<>(4096); + private final ConcurrentHashMap addedPartitions = new ConcurrentHashMap<>(4096); // Locks to control concurrency. Sync operations use write-lock blocking all fetch operations. // For the common-case, we allow concurrent read of single or multiple partitions @@ -81,6 +85,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV private final ReadLock readLock = globalLock.readLock(); private final WriteLock writeLock = globalLock.writeLock(); + private BootstrapIndex bootstrapIndex; + private String getPartitionPathFromFilePath(String fullPath) { return FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), new Path(fullPath).getParent()); } @@ -91,10 +97,11 @@ private String getPartitionPathFromFilePath(String fullPath) { protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) { this.metaClient = metaClient; refreshTimeline(visibleActiveTimeline); - + this.bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient); // Load Pending Compaction Operations resetPendingCompactionOperations(CompactionUtils.getAllPendingCompactionOperations(metaClient).values().stream() .map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue())))); + resetBootstrapBaseFileMapping(Stream.empty()); } /** @@ -117,6 +124,16 @@ protected List addFilesToView(FileStatus[] statuses) { // Group by partition for efficient updates for both InMemory and DiskBased stuctures. fileGroups.stream().collect(Collectors.groupingBy(HoodieFileGroup::getPartitionPath)).forEach((partition, value) -> { if (!isPartitionAvailableInStore(partition)) { + if (bootstrapIndex.useIndex()) { + try (BootstrapIndex.IndexReader reader = bootstrapIndex.createReader()) { + LOG.info("Boostrap Index available for partition " + partition); + List sourceFileMappings = + reader.getSourceFileMappingForPartition(partition); + addBootstrapBaseFileMapping(sourceFileMappings.stream() + .map(s -> new BootstrapBaseFileMapping(new HoodieFileGroupId(s.getPartitionPath(), + s.getFileId()), s.getBoostrapFileStatus()))); + } + } storePartitionView(partition, value); } }); @@ -163,6 +180,7 @@ protected List buildFileGroups(Stream baseFileS if (logFiles.containsKey(pair)) { logFiles.get(pair).forEach(group::addLogFile); } + if (addPendingCompactionFileSlice) { Option> pendingCompaction = getPendingCompactionOperationWithInstant(group.getFileGroupId()); @@ -189,6 +207,8 @@ public final void reset() { addedPartitions.clear(); resetViewState(); + bootstrapIndex = null; + // Initialize with new Hoodie timeline. init(metaClient, getTimeline()); } finally { @@ -313,6 +333,41 @@ protected FileSlice filterBaseFileAfterPendingCompaction(FileSlice fileSlice) { return fileSlice; } + protected HoodieFileGroup addBootstrapBaseFileIfPresent(HoodieFileGroup fileGroup) { + boolean hasBootstrapBaseFile = fileGroup.getAllFileSlices() + .anyMatch(fs -> fs.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)); + if (hasBootstrapBaseFile) { + HoodieFileGroup newFileGroup = new HoodieFileGroup(fileGroup); + newFileGroup.getAllFileSlices().filter(fs -> fs.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)) + .forEach(fs -> fs.setBaseFile( + addBootstrapBaseFileIfPresent(fs.getFileGroupId(), fs.getBaseFile().get()))); + return newFileGroup; + } + return fileGroup; + } + + protected FileSlice addBootstrapBaseFileIfPresent(FileSlice fileSlice) { + if (fileSlice.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)) { + FileSlice copy = new FileSlice(fileSlice); + copy.getBaseFile().ifPresent(dataFile -> { + Option edf = getBootstrapBaseFile(copy.getFileGroupId()); + edf.ifPresent(e -> dataFile.setBootstrapBaseFile(e.getBootstrapBaseFile())); + }); + return copy; + } + return fileSlice; + } + + protected HoodieBaseFile addBootstrapBaseFileIfPresent(HoodieFileGroupId fileGroupId, HoodieBaseFile baseFile) { + if (baseFile.getCommitTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)) { + HoodieBaseFile copy = new HoodieBaseFile(baseFile); + Option edf = getBootstrapBaseFile(fileGroupId); + edf.ifPresent(e -> copy.setBootstrapBaseFile(e.getBootstrapBaseFile())); + return copy; + } + return baseFile; + } + @Override public final Stream> getPendingCompactionOperations() { try { @@ -329,7 +384,8 @@ public final Stream getLatestBaseFiles(String partitionStr) { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestBaseFiles(partitionPath); + return fetchLatestBaseFiles(partitionPath) + .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df)); } finally { readLock.unlock(); } @@ -356,7 +412,8 @@ public final Stream getLatestBaseFilesBeforeOrOn(String partitio .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, maxCommitTime )) .filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst())) - .filter(Option::isPresent).map(Option::get); + .filter(Option::isPresent).map(Option::get) + .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df)); } finally { readLock.unlock(); } @@ -369,9 +426,9 @@ public final Option getBaseFileOn(String partitionStr, String in String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles() - .filter( - baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS, instantTime)) - .filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst().orElse(null)); + .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS, + instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst().orElse(null)) + .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df)); } finally { readLock.unlock(); } @@ -386,7 +443,8 @@ public final Option getLatestBaseFile(String partitionStr, Strin readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestBaseFile(partitionPath, fileId); + return fetchLatestBaseFile(partitionPath, fileId) + .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df)); } finally { readLock.unlock(); } @@ -396,9 +454,10 @@ public final Option getLatestBaseFile(String partitionStr, Strin public final Stream getLatestBaseFilesInRange(List commitsToReturn) { try { readLock.lock(); - return fetchAllStoredFileGroups().map(fileGroup -> Option.fromJavaOptional( + return fetchAllStoredFileGroups().map(fileGroup -> Pair.of(fileGroup.getFileGroupId(), Option.fromJavaOptional( fileGroup.getAllBaseFiles().filter(baseFile -> commitsToReturn.contains(baseFile.getCommitTime()) - && !isBaseFileDueToPendingCompaction(baseFile)).findFirst())).filter(Option::isPresent).map(Option::get); + && !isBaseFileDueToPendingCompaction(baseFile)).findFirst()))).filter(p -> p.getValue().isPresent()) + .map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get())); } finally { readLock.unlock(); } @@ -412,7 +471,8 @@ public final Stream getAllBaseFiles(String partitionStr) { ensurePartitionLoadedCorrectly(partitionPath); return fetchAllBaseFiles(partitionPath) .filter(df -> visibleCommitsAndCompactionTimeline.containsOrBeforeTimelineStarts(df.getCommitTime())) - .filter(df -> !isBaseFileDueToPendingCompaction(df)); + .filter(df -> !isBaseFileDueToPendingCompaction(df)) + .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df)); } finally { readLock.unlock(); } @@ -424,7 +484,8 @@ public final Stream getLatestFileSlices(String partitionStr) { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestFileSlices(partitionPath).map(this::filterBaseFileAfterPendingCompaction); + return fetchLatestFileSlices(partitionPath).map(this::filterBaseFileAfterPendingCompaction) + .map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -440,7 +501,7 @@ public final Option getLatestFileSlice(String partitionStr, String fi String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); Option fs = fetchLatestFileSlice(partitionPath, fileId); - return fs.map(this::filterBaseFileAfterPendingCompaction); + return fs.map(this::filterBaseFileAfterPendingCompaction).map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -462,7 +523,7 @@ public final Stream getLatestUnCompactedFileSlices(String partitionSt return fileGroup.getLatestFileSliceBefore(compactionInstantTime); } return Option.of(fileSlice); - }).map(Option::get); + }).map(Option::get).map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -477,9 +538,10 @@ public final Stream getLatestFileSlicesBeforeOrOn(String partitionStr ensurePartitionLoadedCorrectly(partitionPath); Stream fileSliceStream = fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime); if (includeFileSlicesInPendingCompaction) { - return fileSliceStream.map(this::filterBaseFileAfterPendingCompaction); + return fileSliceStream.map(this::filterBaseFileAfterPendingCompaction).map(this::addBootstrapBaseFileIfPresent); } else { - return fileSliceStream.filter(fs -> !isPendingCompactionScheduledForFileId(fs.getFileGroupId())); + return fileSliceStream.filter(fs -> !isPendingCompactionScheduledForFileId(fs.getFileGroupId())) + .map(this::addBootstrapBaseFileIfPresent); } } finally { readLock.unlock(); @@ -499,7 +561,7 @@ public final Stream getLatestMergedFileSlicesBeforeOrOn(String partit fileSlice = Option.of(fetchMergedFileSlice(fileGroup, fileSlice.get())); } return fileSlice; - }).filter(Option::isPresent).map(Option::get); + }).filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -509,7 +571,7 @@ public final Stream getLatestMergedFileSlicesBeforeOrOn(String partit public final Stream getLatestFileSliceInRange(List commitsToReturn) { try { readLock.lock(); - return fetchLatestFileSliceInRange(commitsToReturn); + return fetchLatestFileSliceInRange(commitsToReturn).map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -521,7 +583,7 @@ public final Stream getAllFileSlices(String partitionStr) { readLock.lock(); String partition = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partition); - return fetchAllFileSlices(partition); + return fetchAllFileSlices(partition).map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -543,7 +605,7 @@ public final Stream getAllFileGroups(String partitionStr) { // in other places. String partition = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partition); - return fetchAllStoredFileGroups(partition); + return fetchAllStoredFileGroups(partition).map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -593,6 +655,48 @@ protected abstract Option> getPendingCompactio */ abstract Stream> fetchPendingCompactionOperations(); + /** + * Check if there is an bootstrap base file present for this file. + * + * @param fgId File-Group Id + * @return true if there is associated bootstrap base-file, false otherwise + */ + protected abstract boolean isBootstrapBaseFilePresentForFileId(HoodieFileGroupId fgId); + + /** + * Resets the bootstrap base file stream and overwrite with the new list. + * + * @param bootstrapBaseFileStream bootstrap Base File Stream + */ + abstract void resetBootstrapBaseFileMapping(Stream bootstrapBaseFileStream); + + /** + * Add bootstrap base file stream to store. + * + * @param bootstrapBaseFileStream bootstrap Base File Stream to be added + */ + abstract void addBootstrapBaseFileMapping(Stream bootstrapBaseFileStream); + + /** + * Remove bootstrap base file stream from store. + * + * @param bootstrapBaseFileStream bootstrap Base File Stream to be removed + */ + abstract void removeBootstrapBaseFileMapping(Stream bootstrapBaseFileStream); + + /** + * Return pending compaction operation for a file-group. + * + * @param fileGroupId File-Group Id + */ + protected abstract Option getBootstrapBaseFile(HoodieFileGroupId fileGroupId); + + /** + * Fetch all bootstrap data files. + */ + abstract Stream fetchBootstrapBaseFiles(); + + /** * Checks if partition is pre-loaded and available in store. * @@ -635,7 +739,7 @@ protected abstract Option> getPendingCompactio */ Stream fetchLatestFileSliceInRange(List commitsToReturn) { return fetchAllStoredFileGroups().map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn)) - .map(Option::get); + .map(Option::get).map(this::addBootstrapBaseFileIfPresent); } /** @@ -645,16 +749,18 @@ Stream fetchLatestFileSliceInRange(List commitsToReturn) { * @return file-slice stream */ Stream fetchAllFileSlices(String partitionPath) { - return fetchAllStoredFileGroups(partitionPath).map(HoodieFileGroup::getAllFileSlices) - .flatMap(sliceList -> sliceList); + return fetchAllStoredFileGroups(partitionPath).map(this::addBootstrapBaseFileIfPresent) + .map(HoodieFileGroup::getAllFileSlices).flatMap(sliceList -> sliceList); } /** * Default implementation for fetching latest base-files for the partition-path. */ Stream fetchLatestBaseFiles(final String partitionPath) { - return fetchAllStoredFileGroups(partitionPath).map(this::getLatestBaseFile).filter(Option::isPresent) - .map(Option::get); + return fetchAllStoredFileGroups(partitionPath) + .map(fg -> Pair.of(fg.getFileGroupId(), getLatestBaseFile(fg))) + .filter(p -> p.getValue().isPresent()) + .map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get())); } protected Option getLatestBaseFile(HoodieFileGroup fileGroup) { @@ -663,10 +769,12 @@ protected Option getLatestBaseFile(HoodieFileGroup fileGroup) { } /** - * Default implementation for fetching latest base-files across all partitions. + * Fetch latest base-files across all partitions. */ - Stream fetchLatestBaseFiles() { - return fetchAllStoredFileGroups().map(this::getLatestBaseFile).filter(Option::isPresent).map(Option::get); + private Stream fetchLatestBaseFiles() { + return fetchAllStoredFileGroups().map(fg -> Pair.of(fg.getFileGroupId(), getLatestBaseFile(fg))) + .filter(p -> p.getValue().isPresent()) + .map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get())); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java index 5e21e43fc88c9..434f873bf6d40 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java @@ -41,6 +41,8 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig { public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = "hoodie.filesystem.view.spillable.mem"; public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION = "hoodie.filesystem.view.spillable.compaction.mem.fraction"; + public static final String FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION = + "hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction"; private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path"; public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY; @@ -53,6 +55,7 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig { public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/"; private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01; + private static final Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = 0.05; private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB /** @@ -90,7 +93,7 @@ public Integer getRemoteViewServerPort() { public long getMaxMemoryForFileGroupMap() { long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM)); - return totalMemory - getMaxMemoryForPendingCompaction(); + return totalMemory - getMaxMemoryForPendingCompaction() - getMaxMemoryForBootstrapBaseFile(); } public long getMaxMemoryForPendingCompaction() { @@ -99,6 +102,14 @@ public long getMaxMemoryForPendingCompaction() { .longValue(); } + public long getMaxMemoryForBootstrapBaseFile() { + long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM)); + long reservedForExternalDataFile = + new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION))) + .longValue(); + return reservedForExternalDataFile; + } + public String getBaseStoreDir() { return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR); } @@ -169,6 +180,11 @@ public Builder withMemFractionForPendingCompaction(Double memFractionForPendingC return this; } + public Builder withMemFractionForExternalDataFile(Double memFractionForExternalDataFile) { + props.setProperty(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, memFractionForExternalDataFile.toString()); + return this; + } + public Builder withBaseStoreDir(String baseStorePath) { props.setProperty(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath); return this; @@ -202,6 +218,8 @@ public FileSystemViewStorageConfig build() { DEFAULT_MAX_MEMORY_FOR_VIEW.toString()); setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION), FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION.toString()); + setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION), + FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE.toString()); setDefaultOnCondition(props, !props.containsKey(ROCKSDB_BASE_PATH_PROP), ROCKSDB_BASE_PATH_PROP, DEFAULT_ROCKSDB_BASE_PATH); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index 56ae22d5cd43a..4cbd2e3866ca1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.view; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.BootstrapBaseFileMapping; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -58,6 +59,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem */ protected Map> fgIdToPendingCompaction; + /** + * PartitionPath + File-Id to bootstrap base File (Index Only bootstrapped). + */ + protected Map fgIdToBootstrapBaseFile; + /** * Flag to determine if closed. */ @@ -99,6 +105,7 @@ public void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveT protected void resetViewState() { this.fgIdToPendingCompaction = null; this.partitionToFileGroupsMap = null; + this.fgIdToBootstrapBaseFile = null; } protected Map> createPartitionToFileGroups() { @@ -110,6 +117,11 @@ protected Map> createFileId return fileIdToPendingCompaction; } + protected Map createFileIdToBootstrapBaseFileMap( + Map fileGroupIdBootstrapBaseFileMap) { + return fileGroupIdBootstrapBaseFileMap; + } + /** * Create a file system view, as of the given timeline, with the provided file statuses. */ @@ -185,6 +197,48 @@ Stream> fetchPendingCompactionOperations() { } + @Override + protected boolean isBootstrapBaseFilePresentForFileId(HoodieFileGroupId fgId) { + return fgIdToBootstrapBaseFile.containsKey(fgId); + } + + @Override + void resetBootstrapBaseFileMapping(Stream bootstrapBaseFileStream) { + // Build fileId to bootstrap Data File + this.fgIdToBootstrapBaseFile = createFileIdToBootstrapBaseFileMap(bootstrapBaseFileStream + .collect(Collectors.toMap(BootstrapBaseFileMapping::getFileGroupId, x -> x))); + } + + @Override + void addBootstrapBaseFileMapping(Stream bootstrapBaseFileStream) { + bootstrapBaseFileStream.forEach(bootstrapBaseFile -> { + ValidationUtils.checkArgument(!fgIdToBootstrapBaseFile.containsKey(bootstrapBaseFile.getFileGroupId()), + "Duplicate FileGroupId found in bootstrap base file mapping. FgId :" + + bootstrapBaseFile.getFileGroupId()); + fgIdToBootstrapBaseFile.put(bootstrapBaseFile.getFileGroupId(), bootstrapBaseFile); + }); + } + + @Override + void removeBootstrapBaseFileMapping(Stream bootstrapBaseFileStream) { + bootstrapBaseFileStream.forEach(bootstrapBaseFile -> { + ValidationUtils.checkArgument(fgIdToBootstrapBaseFile.containsKey(bootstrapBaseFile.getFileGroupId()), + "Trying to remove a FileGroupId which is not found in bootstrap base file mapping. FgId :" + + bootstrapBaseFile.getFileGroupId()); + fgIdToBootstrapBaseFile.remove(bootstrapBaseFile.getFileGroupId()); + }); + } + + @Override + protected Option getBootstrapBaseFile(HoodieFileGroupId fileGroupId) { + return Option.ofNullable(fgIdToBootstrapBaseFile.get(fileGroupId)); + } + + @Override + Stream fetchBootstrapBaseFiles() { + return fgIdToBootstrapBaseFile.values().stream(); + } + @Override protected Option> getPendingCompactionOperationWithInstant(HoodieFileGroupId fgId) { return Option.ofNullable(fgIdToPendingCompaction.get(fgId)); @@ -213,6 +267,7 @@ public void close() { super.reset(); partitionToFileGroupsMap = null; fgIdToPendingCompaction = null; + fgIdToBootstrapBaseFile = null; } @Override 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 6446a6cf3b929..2e31cea6e7ea8 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 @@ -91,7 +91,6 @@ protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) { LOG.error("Got exception trying to perform incremental sync. Reverting to complete sync", ioe); } - LOG.warn("Incremental Sync of timeline is turned off or deemed unsafe. Will revert to full syncing"); super.runSync(oldTimeline, newTimeline); } 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 00172d714ae5a..29309a57d11ed 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 @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.view; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.BootstrapBaseFileMapping; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; @@ -267,6 +268,62 @@ Stream fetchAllBaseFiles(String partitionPath) { schemaHelper.getPrefixForDataFileViewByPartition(partitionPath)).map(Pair::getValue); } + @Override + protected boolean isBootstrapBaseFilePresentForFileId(HoodieFileGroupId fgId) { + return getBootstrapBaseFile(fgId).isPresent(); + } + + @Override + void resetBootstrapBaseFileMapping(Stream bootstrapBaseFileStream) { + rocksDB.writeBatch(batch -> { + bootstrapBaseFileStream.forEach(externalBaseFile -> { + rocksDB.putInBatch(batch, schemaHelper.getColFamilyForBootstrapBaseFile(), + schemaHelper.getKeyForBootstrapBaseFile(externalBaseFile.getFileGroupId()), externalBaseFile); + }); + LOG.info("Initializing external data file mapping. Count=" + batch.count()); + }); + } + + @Override + void addBootstrapBaseFileMapping(Stream bootstrapBaseFileStream) { + rocksDB.writeBatch(batch -> { + bootstrapBaseFileStream.forEach(externalBaseFile -> { + ValidationUtils.checkArgument(!isBootstrapBaseFilePresentForFileId(externalBaseFile.getFileGroupId()), + "Duplicate FileGroupId found in external data file. FgId :" + externalBaseFile.getFileGroupId()); + rocksDB.putInBatch(batch, schemaHelper.getColFamilyForBootstrapBaseFile(), + schemaHelper.getKeyForBootstrapBaseFile(externalBaseFile.getFileGroupId()), externalBaseFile); + }); + }); + } + + @Override + void removeBootstrapBaseFileMapping(Stream bootstrapBaseFileStream) { + rocksDB.writeBatch(batch -> { + bootstrapBaseFileStream.forEach(externalBaseFile -> { + ValidationUtils.checkArgument( + getBootstrapBaseFile(externalBaseFile.getFileGroupId()) != null, + "Trying to remove a FileGroupId which is not found in external data file mapping. FgId :" + + externalBaseFile.getFileGroupId()); + rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForBootstrapBaseFile(), + schemaHelper.getKeyForBootstrapBaseFile(externalBaseFile.getFileGroupId())); + }); + }); + } + + @Override + protected Option getBootstrapBaseFile(HoodieFileGroupId fileGroupId) { + String lookupKey = schemaHelper.getKeyForBootstrapBaseFile(fileGroupId); + BootstrapBaseFileMapping externalBaseFile = + rocksDB.get(schemaHelper.getColFamilyForBootstrapBaseFile(), lookupKey); + return Option.ofNullable(externalBaseFile); + } + + @Override + Stream fetchBootstrapBaseFiles() { + return rocksDB.prefixSearch(schemaHelper.getColFamilyForBootstrapBaseFile(), "") + .map(Pair::getValue); + } + @Override Stream fetchAllStoredFileGroups(String partitionPath) { return getFileGroups(rocksDB.prefixSearch(schemaHelper.getColFamilyForView(), diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java index 4ae4ee4442e3e..2e136ea5ca146 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java @@ -18,7 +18,14 @@ package org.apache.hudi.common.table.view; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; +import org.apache.hadoop.fs.FileStatus; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.BootstrapBaseFileMapping; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -26,17 +33,9 @@ import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; - -import org.apache.hadoop.fs.FileStatus; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.stream.Stream; - /** * Table FileSystemView implementation where view is stored in spillable disk using fixed memory. */ @@ -46,6 +45,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView { private final long maxMemoryForFileGroupMap; private final long maxMemoryForPendingCompaction; + private final long maxMemoryForBootstrapBaseFile; private final String baseStoreDir; public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline, @@ -53,6 +53,7 @@ public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieT super(config.isIncrementalTimelineSyncEnabled()); this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap(); this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction(); + this.maxMemoryForBootstrapBaseFile = config.getMaxMemoryForBootstrapBaseFile(); this.baseStoreDir = config.getBaseStoreDir(); init(metaClient, visibleActiveTimeline); } @@ -92,6 +93,22 @@ protected Map> createFileId } } + @Override + protected Map createFileIdToBootstrapBaseFileMap( + Map fileGroupIdBootstrapBaseFileMap) { + try { + LOG.info("Creating bootstrap base File Map using external spillable Map. Max Mem=" + maxMemoryForBootstrapBaseFile + + ", BaseDir=" + baseStoreDir); + new File(baseStoreDir).mkdirs(); + Map pendingMap = new ExternalSpillableMap<>( + maxMemoryForBootstrapBaseFile, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>()); + pendingMap.putAll(fileGroupIdBootstrapBaseFileMap); + return pendingMap; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + @Override public Stream getAllFileGroups() { return ((ExternalSpillableMap) partitionToFileGroupsMap).valueStream() @@ -101,7 +118,11 @@ public Stream getAllFileGroups() { @Override Stream> fetchPendingCompactionOperations() { return ((ExternalSpillableMap) fgIdToPendingCompaction).valueStream(); + } + @Override + Stream fetchBootstrapBaseFiles() { + return ((ExternalSpillableMap) fgIdToBootstrapBaseFile).valueStream(); } @Override 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 cf233d67112b4..f3e4dc62837c1 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 @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieFileGroupId; @@ -72,6 +73,8 @@ public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, builder.setDeltaFilePaths(fileSlice.getLogFiles().map(lf -> lf.getPath().getName()).collect(Collectors.toList())); if (fileSlice.getBaseFile().isPresent()) { builder.setDataFilePath(fileSlice.getBaseFile().get().getFileName()); + builder.setBootstrapFilePath(fileSlice.getBaseFile().get().getBootstrapBaseFile().map(BaseFile::getPath) + .orElse(null)); } if (metricsCaptureFunction.isPresent()) { @@ -106,6 +109,7 @@ public static HoodieCompactionPlan buildFromFileSlices(List readParquetFooter(Configuration configuration } public static Schema readAvroSchema(Configuration configuration, Path parquetFilePath) { - return new AvroSchemaConverter().convert(readSchema(configuration, parquetFilePath)); + return new AvroSchemaConverter(configuration).convert(readSchema(configuration, parquetFilePath)); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java index 7207dd81dae4e..e56dba41adfb2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java @@ -45,16 +45,19 @@ public class RocksDBSchemaHelper { private final String colFamilyForView; private final String colFamilyForPendingCompaction; + private final String colFamilyForBootstrapBaseFile; private final String colFamilyForStoredPartitions; public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) { + this.colFamilyForBootstrapBaseFile = "hudi_bootstrap_basefile_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForPendingCompaction = "hudi_pending_compaction_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForStoredPartitions = "hudi_partitions_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForView = "hudi_view_" + metaClient.getBasePath().replace("/", "_"); } public List getAllColumnFamilies() { - return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForStoredPartitions()); + return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForBootstrapBaseFile(), + getColFamilyForStoredPartitions()); } public String getKeyForPartitionLookup(String partition) { @@ -65,6 +68,10 @@ public String getKeyForPendingCompactionLookup(HoodieFileGroupId fgId) { return getPartitionFileIdBasedLookup(fgId); } + public String getKeyForBootstrapBaseFile(HoodieFileGroupId fgId) { + return getPartitionFileIdBasedLookup(fgId); + } + public String getKeyForSliceView(HoodieFileGroup fileGroup, FileSlice slice) { return getKeyForSliceView(fileGroup.getPartitionPath(), fileGroup.getFileGroupId().getFileId(), slice.getBaseInstantTime()); @@ -111,6 +118,10 @@ public String getColFamilyForPendingCompaction() { return colFamilyForPendingCompaction; } + public String getColFamilyForBootstrapBaseFile() { + return colFamilyForBootstrapBaseFile; + } + public String getColFamilyForStoredPartitions() { return colFamilyForStoredPartitions; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java index d1e33056de8aa..49f1075508c86 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java @@ -81,6 +81,10 @@ public static String nullToEmpty(@Nullable String string) { return string == null ? "" : string; } + public static String objToString(@Nullable Object obj) { + return obj == null ? null : obj.toString(); + } + /** * Returns the given string if it is nonempty; {@code null} otherwise. * diff --git a/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java b/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java new file mode 100644 index 0000000000000..d0dfd939f6548 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.bootstrap; + +import org.apache.hudi.avro.model.HoodieFSPermission; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.avro.model.HoodiePath; +import org.apache.hudi.common.bootstrap.index.BootstrapIndex; +import org.apache.hudi.common.bootstrap.index.BootstrapIndex.IndexWriter; +import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex; +import org.apache.hudi.common.model.BootstrapFileMapping; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.hadoop.fs.permission.FsAction; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** + * Unit Tests for Bootstrap Index. + */ +public class TestBootstrapIndex extends HoodieCommonTestHarness { + + private static String[] PARTITIONS = {"2020/03/18", "2020/03/19", "2020/03/20", "2020/03/21"}; + private static String BOOTSTRAP_BASE_PATH = "/tmp/source/parquet_tables/table1"; + + @BeforeEach + + public void init() throws IOException { + initMetaClient(); + } + + @Test + public void testBootstrapIndex() throws IOException { + testBootstrapIndexOneRound(10); + } + + @Test + public void testBootstrapIndexRecreateIndex() throws IOException { + testBootstrapIndexOneRound(10); + + HFileBootstrapIndex index = new HFileBootstrapIndex(metaClient); + index.dropIndex(); + + // Run again this time recreating bootstrap index + testBootstrapIndexOneRound(5); + } + + @Test + public void testBootstrapIndexConcurrent() throws Exception { + Map> bootstrapMapping = generateBootstrapIndex(100); + final int numThreads = 20; + final int numRequestsPerThread = 50; + ExecutorService service = Executors.newFixedThreadPool(numThreads); + try { + List> futureList = new ArrayList<>(); + for (int i = 0; i < numThreads; i++) { + Future result = service.submit(() -> { + for (int j = 0; j < numRequestsPerThread; j++) { + validateBootstrapIndex(bootstrapMapping); + } + return true; + }); + futureList.add(result); + } + + for (Future res : futureList) { + res.get(); + } + } finally { + service.shutdownNow(); + } + } + + private void testBootstrapIndexOneRound(int numEntriesPerPartition) throws IOException { + Map> bootstrapMapping = generateBootstrapIndex(numEntriesPerPartition); + validateBootstrapIndex(bootstrapMapping); + } + + private Map> generateBootstrapIndex(int numEntriesPerPartition) + throws IOException { + Map> bootstrapMapping = generateBootstrapMapping(numEntriesPerPartition); + BootstrapIndex index = new HFileBootstrapIndex(metaClient); + try (IndexWriter writer = index.createWriter(BOOTSTRAP_BASE_PATH)) { + writer.begin(); + bootstrapMapping.entrySet().stream().forEach(e -> writer.appendNextPartition(e.getKey(), e.getValue())); + writer.finish(); + } + return bootstrapMapping; + } + + private void validateBootstrapIndex(Map> bootstrapMapping) { + BootstrapIndex index = new HFileBootstrapIndex(metaClient); + try (BootstrapIndex.IndexReader reader = index.createReader()) { + List partitions = reader.getIndexedPartitionPaths(); + assertEquals(bootstrapMapping.size(), partitions.size()); + long expNumFileGroupKeys = bootstrapMapping.values().stream().flatMap(x -> x.stream()).count(); + long gotNumFileGroupKeys = reader.getIndexedFileIds().size(); + assertEquals(expNumFileGroupKeys, gotNumFileGroupKeys); + + bootstrapMapping.entrySet().stream().forEach(e -> { + List gotMapping = reader.getSourceFileMappingForPartition(e.getKey()); + List expected = new ArrayList<>(e.getValue()); + Collections.sort(gotMapping); + Collections.sort(expected); + assertEquals(expected, gotMapping, "Check for bootstrap index entries for partition " + e.getKey()); + List fileIds = e.getValue().stream().map(BootstrapFileMapping::getFileGroupId) + .collect(Collectors.toList()); + Map lookupResult = reader.getSourceFileMappingForFileIds(fileIds); + assertEquals(fileIds.size(), lookupResult.size()); + e.getValue().forEach(x -> { + BootstrapFileMapping res = lookupResult.get(x.getFileGroupId()); + assertNotNull(res); + assertEquals(x.getFileId(), res.getFileId()); + assertEquals(x.getPartitionPath(), res.getPartitionPath()); + assertEquals(BOOTSTRAP_BASE_PATH, res.getBootstrapBasePath()); + assertEquals(x.getBoostrapFileStatus(), res.getBoostrapFileStatus()); + assertEquals(x.getBootstrapPartitionPath(), res.getBootstrapPartitionPath()); + }); + }); + } + } + + private Map> generateBootstrapMapping(int numEntriesPerPartition) { + return Arrays.stream(PARTITIONS).map(partition -> { + return Pair.of(partition, IntStream.range(0, numEntriesPerPartition).mapToObj(idx -> { + String hudiFileId = UUID.randomUUID().toString(); + String sourceFileName = idx + ".parquet"; + HoodieFileStatus sourceFileStatus = HoodieFileStatus.newBuilder() + .setPath(HoodiePath.newBuilder().setUri(BOOTSTRAP_BASE_PATH + "/" + partition + "/" + sourceFileName).build()) + .setLength(256 * 1024 * 1024L) + .setAccessTime(new Date().getTime()) + .setModificationTime(new Date().getTime() + 99999) + .setBlockReplication(2) + .setOwner("hudi") + .setGroup("hudi") + .setBlockSize(128 * 1024 * 1024L) + .setPermission(HoodieFSPermission.newBuilder().setUserAction(FsAction.ALL.name()) + .setGroupAction(FsAction.READ.name()).setOtherAction(FsAction.NONE.name()).setStickyBit(true).build()) + .build(); + return new BootstrapFileMapping(BOOTSTRAP_BASE_PATH, partition, partition, sourceFileStatus, hudiFileId); + }).collect(Collectors.toList())); + }).collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + } +} 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 5e833791067cb..3914c813f00eb 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 @@ -19,7 +19,15 @@ package org.apache.hudi.common.table.view; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieFSPermission; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.avro.model.HoodiePath; +import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.bootstrap.index.BootstrapIndex.IndexWriter; +import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.BootstrapFileMapping; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; @@ -35,22 +43,29 @@ import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.table.view.TableFileSystemView.SliceView; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.io.IOException; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; + +import java.util.Date; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -70,16 +85,23 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness { private static final Logger LOG = LogManager.getLogger(TestHoodieTableFileSystemView.class); + private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with bootstrap enable={0}"; - private static String TEST_WRITE_TOKEN = "1-0-1"; + private static final String TEST_WRITE_TOKEN = "1-0-1"; + private static final String BOOTSTRAP_SOURCE_PATH = "/usr/warehouse/hive/data/tables/src1/"; protected SyncableFileSystemView fsView; protected BaseFileOnlyView roView; protected SliceView rtView; + public static Stream configParams() { + return Arrays.stream(new Boolean[][] {{true}, {false}}).map(Arguments::of); + } + @BeforeEach - public void init() throws IOException { - initMetaClient(); + public void setup() throws IOException { + metaClient = HoodieTestUtils.init(tempDir.toAbsolutePath().toString(), getTableType(), BOOTSTRAP_SOURCE_PATH); + basePath = metaClient.getBasePath(); refreshFsView(); } @@ -128,6 +150,7 @@ protected void testViewForFileSlicesWithNoBaseFile(int expNumTotalFileSlices, in Paths.get(basePath, partitionPath, fileName1).toFile().createNewFile(); Paths.get(basePath, partitionPath, fileName2).toFile().createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); @@ -184,29 +207,33 @@ protected void testViewForFileSlicesWithNoBaseFile(int expNumTotalFileSlices, in "Total number of file-groups in view matches expected"); } - @Test - public void testViewForFileSlicesWithNoBaseFileAndRequestedCompaction() throws Exception { - testViewForFileSlicesWithAsyncCompaction(true, false, 2, 1, true); + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testViewForFileSlicesWithNoBaseFileAndRequestedCompaction(boolean testBootstrap) throws Exception { + testViewForFileSlicesWithAsyncCompaction(true, false, 2, 1, true, testBootstrap); } - @Test - public void testViewForFileSlicesWithBaseFileAndRequestedCompaction() throws Exception { - testViewForFileSlicesWithAsyncCompaction(false, false, 2, 2, true); + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testViewForFileSlicesWithBaseFileAndRequestedCompaction(boolean testBootstrap) throws Exception { + testViewForFileSlicesWithAsyncCompaction(false, false, 2, 2, true, testBootstrap); } - @Test - public void testViewForFileSlicesWithNoBaseFileAndInflightCompaction() throws Exception { - testViewForFileSlicesWithAsyncCompaction(true, true, 2, 1, true); + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testViewForFileSlicesWithNoBaseFileAndInflightCompaction(boolean testBootstrap) throws Exception { + testViewForFileSlicesWithAsyncCompaction(true, true, 2, 1, true, testBootstrap); } - @Test - public void testViewForFileSlicesWithBaseFileAndInflightCompaction() throws Exception { - testViewForFileSlicesWithAsyncCompaction(false, true, 2, 2, true); + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testViewForFileSlicesWithBaseFileAndInflightCompaction(boolean testBootstrap) throws Exception { + testViewForFileSlicesWithAsyncCompaction(false, true, 2, 2, true, testBootstrap); } /** * Returns all file-slices including uncommitted ones. - * + * * @param partitionPath * @return */ @@ -217,7 +244,7 @@ private Stream getAllRawFileSlices(String partitionPath) { /** * Returns latest raw file-slices including uncommitted ones. - * + * * @param partitionPath * @return */ @@ -226,9 +253,36 @@ public Stream getLatestRawFileSlices(String partitionPath) { .filter(Option::isPresent).map(Option::get); } + private void checkExternalFile(HoodieFileStatus srcFileStatus, Option bootstrapBaseFile, boolean testBootstrap) { + if (testBootstrap) { + assertTrue(bootstrapBaseFile.isPresent()); + assertEquals(FileStatusUtils.toPath(srcFileStatus.getPath()), new Path(bootstrapBaseFile.get().getPath())); + assertEquals(srcFileStatus.getPath(), FileStatusUtils.fromPath(new Path(bootstrapBaseFile.get().getPath()))); + assertEquals(srcFileStatus.getOwner(), bootstrapBaseFile.get().getFileStatus().getOwner()); + assertEquals(srcFileStatus.getGroup(), bootstrapBaseFile.get().getFileStatus().getGroup()); + assertEquals(srcFileStatus.getAccessTime(), new Long(bootstrapBaseFile.get().getFileStatus().getAccessTime())); + assertEquals(srcFileStatus.getModificationTime(), + new Long(bootstrapBaseFile.get().getFileStatus().getModificationTime())); + assertEquals(srcFileStatus.getBlockSize(), new Long(bootstrapBaseFile.get().getFileStatus().getBlockSize())); + assertEquals(srcFileStatus.getLength(), new Long(bootstrapBaseFile.get().getFileStatus().getLen())); + assertEquals(srcFileStatus.getBlockReplication(), + new Integer(bootstrapBaseFile.get().getFileStatus().getReplication())); + assertEquals(srcFileStatus.getIsDir() == null ? false : srcFileStatus.getIsDir(), + bootstrapBaseFile.get().getFileStatus().isDirectory()); + assertEquals(FileStatusUtils.toFSPermission(srcFileStatus.getPermission()), + bootstrapBaseFile.get().getFileStatus().getPermission()); + assertEquals(srcFileStatus.getPermission(), + FileStatusUtils.fromFSPermission(bootstrapBaseFile.get().getFileStatus().getPermission())); + assertEquals(srcFileStatus.getSymlink() != null, + bootstrapBaseFile.get().getFileStatus().isSymlink()); + } else { + assertFalse(bootstrapBaseFile.isPresent()); + } + } + /** * Helper method to test Views in the presence of concurrent compaction. - * + * * @param skipCreatingDataFile if set, first File Slice will not have data-file set. This would simulate inserts going * directly to log files * @param isCompactionInFlight if set, compaction was inflight (running) when view was tested first time, otherwise @@ -236,16 +290,31 @@ public Stream getLatestRawFileSlices(String partitionPath) { * @param expTotalFileSlices Total number of file-slices across file-groups in the partition path * @param expTotalDataFiles Total number of data-files across file-groups in the partition path * @param includeInvalidAndInflight Whether view includes inflight and invalid file-groups. + * @param testBootstrap enable Bootstrap and test * @throws Exception - */ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFile, boolean isCompactionInFlight, - int expTotalFileSlices, int expTotalDataFiles, boolean includeInvalidAndInflight) throws Exception { + int expTotalFileSlices, int expTotalDataFiles, boolean includeInvalidAndInflight, boolean testBootstrap) + throws Exception { String partitionPath = "2016/05/01"; new File(basePath + "/" + partitionPath).mkdirs(); String fileId = UUID.randomUUID().toString(); + String srcName = "part_0000.parquet"; + HoodieFileStatus srcFileStatus = HoodieFileStatus.newBuilder() + .setPath(HoodiePath.newBuilder().setUri(BOOTSTRAP_SOURCE_PATH + partitionPath + "/" + srcName).build()) + .setLength(256 * 1024 * 1024L) + .setAccessTime(new Date().getTime()) + .setModificationTime(new Date().getTime() + 99999) + .setBlockReplication(2) + .setOwner("hudi") + .setGroup("hudi") + .setBlockSize(128 * 1024 * 1024L) + .setPermission(HoodieFSPermission.newBuilder().setUserAction(FsAction.ALL.name()) + .setGroupAction(FsAction.READ.name()).setOtherAction(FsAction.NONE.name()).setStickyBit(true).build()) + .build(); // if skipCreatingDataFile, then instantTime1 below acts like delta-commit, otherwise it is base-commit - String instantTime1 = "1"; + String instantTime1 = testBootstrap && !skipCreatingDataFile ? HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS : "1"; String deltaInstantTime1 = "2"; String deltaInstantTime2 = "3"; @@ -265,12 +334,30 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); + if (testBootstrap && !skipCreatingDataFile) { + try (IndexWriter writer = new HFileBootstrapIndex(metaClient).createWriter(BOOTSTRAP_SOURCE_PATH)) { + writer.begin(); + BootstrapFileMapping mapping = new BootstrapFileMapping(BOOTSTRAP_SOURCE_PATH, partitionPath, + partitionPath, srcFileStatus, fileId); + List b = new ArrayList<>(); + b.add(mapping); + writer.appendNextPartition(partitionPath, b); + writer.finish(); + } + } saveAsComplete(commitTimeline, instant1, Option.empty()); saveAsComplete(commitTimeline, deltaInstant2, Option.empty()); saveAsComplete(commitTimeline, deltaInstant3, Option.empty()); refreshFsView(); List fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList()); + assertEquals(1, fileSlices.size()); + FileSlice fileSlice = fileSlices.get(0); + assertEquals(instantTime1, fileSlice.getBaseInstantTime()); + if (!skipCreatingDataFile) { + assertTrue(fileSlice.getBaseFile().isPresent()); + checkExternalFile(srcFileStatus, fileSlice.getBaseFile().get().getBootstrapBaseFile(), testBootstrap); + } String compactionRequestedTime = "4"; String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId); List> partitionFileSlicesPairs = new ArrayList<>(); @@ -328,10 +415,11 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData List fileSliceList = rtView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); assertEquals(1, fileSliceList.size(), "Expect file-slice to be merged"); - FileSlice fileSlice = fileSliceList.get(0); + fileSlice = fileSliceList.get(0); assertEquals(fileId, fileSlice.getFileId()); if (!skipCreatingDataFile) { assertEquals(dataFileName, fileSlice.getBaseFile().get().getFileName(), "Data file must be present"); + checkExternalFile(srcFileStatus, fileSlice.getBaseFile().get().getBootstrapBaseFile(), testBootstrap); } else { assertFalse(fileSlice.getBaseFile().isPresent(), "No data-file expected as it was not created"); } @@ -364,27 +452,34 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData } else { assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent"); dataFiles.forEach(df -> assertEquals(df.getCommitTime(), instantTime1, "Expect data-file for instant 1 be returned")); + checkExternalFile(srcFileStatus, dataFiles.get(0).getBootstrapBaseFile(), testBootstrap); } + dataFiles = roView.getLatestBaseFiles(partitionPath).collect(Collectors.toList()); if (skipCreatingDataFile) { assertEquals(0, dataFiles.size(), "Expect no data file to be returned"); } else { assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent"); dataFiles.forEach(df -> assertEquals(df.getCommitTime(), instantTime1, "Expect data-file for instant 1 be returned")); + checkExternalFile(srcFileStatus, dataFiles.get(0).getBootstrapBaseFile(), testBootstrap); } + dataFiles = roView.getLatestBaseFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); if (skipCreatingDataFile) { assertEquals(0, dataFiles.size(), "Expect no data file to be returned"); } else { assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent"); dataFiles.forEach(df -> assertEquals(df.getCommitTime(), instantTime1, "Expect data-file for instant 1 be returned")); + checkExternalFile(srcFileStatus, dataFiles.get(0).getBootstrapBaseFile(), testBootstrap); } + dataFiles = roView.getLatestBaseFilesInRange(allInstantTimes).collect(Collectors.toList()); if (skipCreatingDataFile) { assertEquals(0, dataFiles.size(), "Expect no data file to be returned"); } else { assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent"); dataFiles.forEach(df -> assertEquals(df.getCommitTime(), instantTime1, "Expect data-file for instant 1 be returned")); + checkExternalFile(srcFileStatus, dataFiles.get(0).getBootstrapBaseFile(), testBootstrap); } // Inflight/Orphan File-groups needs to be in the view @@ -499,6 +594,7 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData assertEquals(1, dataFiles.size(), "Expect only one data-files in latest view as there is only one file-group"); assertEquals(compactDataFileName, dataFiles.get(0).getFileName(), "Data Filename must match"); assertEquals(1, fileSliceList.size(), "Only one latest file-slice in the partition"); + assertFalse(dataFiles.get(0).getBootstrapBaseFile().isPresent(), "No external data file must be present"); fileSlice = fileSliceList.get(0); assertEquals(fileId, fileSlice.getFileId(), "Check file-Id is set correctly"); assertEquals(compactDataFileName, fileSlice.getBaseFile().get().getFileName(), @@ -513,16 +609,30 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData // Data Files API tests dataFiles = roView.getLatestBaseFiles().collect(Collectors.toList()); assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent"); - dataFiles.forEach(df -> assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned")); + assertFalse(dataFiles.get(0).getBootstrapBaseFile().isPresent(),"No external data file must be present"); + + dataFiles.forEach(df -> { + assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned"); + assertFalse(df.getBootstrapBaseFile().isPresent(), "No external data file must be present"); + }); dataFiles = roView.getLatestBaseFiles(partitionPath).collect(Collectors.toList()); assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent"); - dataFiles.forEach(df -> assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned")); + dataFiles.forEach(df -> { + assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned"); + assertFalse(df.getBootstrapBaseFile().isPresent(), "No external data file must be present"); + }); dataFiles = roView.getLatestBaseFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList()); assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent"); - dataFiles.forEach(df -> assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned")); + dataFiles.forEach(df -> { + assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned"); + assertFalse(df.getBootstrapBaseFile().isPresent(), "No external data file must be present"); + }); dataFiles = roView.getLatestBaseFilesInRange(allInstantTimes).collect(Collectors.toList()); assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent"); - dataFiles.forEach(df -> assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned")); + dataFiles.forEach(df -> { + assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned"); + assertFalse(df.getBootstrapBaseFile().isPresent(), "No external data file must be present"); + }); assertEquals(expTotalFileSlices, rtView.getAllFileSlices(partitionPath).count(), "Total number of file-slices in partitions matches expected"); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java index 31986b9088932..0aad0c2b30edf 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java @@ -43,7 +43,13 @@ public class HoodieCommonTestHarness { * Initializes basePath. */ protected void initPath() { - this.basePath = tempDir.toAbsolutePath().toString(); + try { + java.nio.file.Path basePath = tempDir.resolve("dataset"); + java.nio.file.Files.createDirectories(basePath); + this.basePath = basePath.toString(); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } } /** diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index 841a74afd38ee..6c56ff131510f 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -19,6 +19,9 @@ package org.apache.hudi.common.testutils; +import org.apache.avro.Conversions; +import org.apache.avro.LogicalTypes; +import org.apache.avro.generic.GenericFixed; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.fs.FSUtils; @@ -34,12 +37,9 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; -import org.apache.avro.Conversions; -import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.generic.GenericArray; import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -64,6 +64,7 @@ import java.util.Random; import java.util.Set; import java.util.UUID; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -107,7 +108,7 @@ public class HoodieTestDataGenerator { + "{\"name\": \"nation\", \"type\": \"bytes\"}," + "{\"name\": \"user_defined_delete_marker_field\", \"type\": \"boolean\", \"default\": false}," + "{\"name\":\"current_date\",\"type\": {\"type\": \"int\", \"logicalType\": \"date\"}}," - + "{\"name\":\"current_ts\",\"type\": {\"type\": \"long\", \"logicalType\": \"timestamp-micros\"}}," + + "{\"name\":\"current_ts\",\"type\": {\"type\": \"long\"}}," + "{\"name\":\"height\",\"type\":{\"type\":\"fixed\",\"name\":\"abc\",\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":6}},"; public static final String TRIP_EXAMPLE_SCHEMA = @@ -126,6 +127,7 @@ public class HoodieTestDataGenerator { public static final String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double,int,bigint,float,binary,boolean,int,bigint,decimal(10,6)," + "map,struct,array>,boolean"; + public static final Schema AVRO_SCHEMA = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA); public static final Schema AVRO_SCHEMA_WITH_METADATA_FIELDS = HoodieAvroUtils.addMetadataFields(AVRO_SCHEMA); @@ -404,7 +406,7 @@ public static void createSavepointFile(String basePath, String instantTime, Conf } public List generateInsertsAsPerSchema(String commitTime, Integer n, String schemaStr) { - return generateInsertsStream(commitTime, n, schemaStr).collect(Collectors.toList()); + return generateInsertsStream(commitTime, n, false, schemaStr).collect(Collectors.toList()); } /** @@ -431,38 +433,35 @@ public List generateInserts(String instantTime, Integer n, boolean /** * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys. */ - public Stream generateInsertsStream(String commitTime, Integer n, String schemaStr) { - return generateInsertsStream(commitTime, n, false, schemaStr); + public Stream generateInsertsStream(String commitTime, Integer n, boolean isFlattened, String schemaStr) { + return generateInsertsStream(commitTime, n, isFlattened, schemaStr, false); } public List generateInsertsContainsAllPartitions(String instantTime, Integer n) { if (n < partitionPaths.length) { throw new HoodieIOException("n must greater then partitionPaths length"); } - return generateInsertsStream( - instantTime, n, false, TRIP_EXAMPLE_SCHEMA, true).collect(Collectors.toList()); + return generateInsertsStream(instantTime, n, false, TRIP_EXAMPLE_SCHEMA, true).collect(Collectors.toList()); } - /** - * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys. - */ - public Stream generateInsertsStream( - String instantTime, Integer n, boolean isFlattened, String schemaStr) { - return generateInsertsStream(instantTime, n, isFlattened, schemaStr, false); + public Stream generateInsertsStream(String commitTime, Integer n, boolean isFlattened, String schemaStr, boolean containsAllPartitions) { + return generateInsertsStream(commitTime, n, isFlattened, schemaStr, containsAllPartitions, + () -> partitionPaths[RAND.nextInt(partitionPaths.length)], + () -> UUID.randomUUID().toString()); } /** * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys. */ - public Stream generateInsertsStream( - String instantTime, Integer n, boolean isFlattened, String schemaStr, boolean containsAllPartitions) { + public Stream generateInsertsStream(String instantTime, Integer n, boolean isFlattened, String schemaStr, boolean containsAllPartitions, + Supplier partitionPathSupplier, Supplier recordKeySupplier) { int currSize = getNumExistingKeys(schemaStr); return IntStream.range(0, n).boxed().map(i -> { - String partitionPath = partitionPaths[RAND.nextInt(partitionPaths.length)]; + String partitionPath = partitionPathSupplier.get(); if (containsAllPartitions && i < partitionPaths.length) { partitionPath = partitionPaths[i]; } - HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath); + HoodieKey key = new HoodieKey(recordKeySupplier.get(), partitionPath); KeyPartition kp = new KeyPartition(); kp.key = key; kp.partitionPath = partitionPath; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java index fb283d41c1461..92d431c2df2dd 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java @@ -118,6 +118,12 @@ public static HoodieTableMetaClient init(String basePath, HoodieTableType tableT return init(getDefaultHadoopConf(), basePath, tableType); } + public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType, String bootstrapBasePath) throws IOException { + Properties props = new Properties(); + props.setProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH, bootstrapBasePath); + return init(getDefaultHadoopConf(), basePath, tableType, props); + } + public static HoodieTableMetaClient init(String basePath, HoodieFileFormat baseFileFormat) throws IOException { return init(getDefaultHadoopConf(), basePath, HoodieTableType.COPY_ON_WRITE, baseFileFormat); } diff --git a/hudi-common/src/test/resources/log4j-surefire-quiet.properties b/hudi-common/src/test/resources/log4j-surefire-quiet.properties index b21b5d4070c41..ca0a50c84270c 100644 --- a/hudi-common/src/test/resources/log4j-surefire-quiet.properties +++ b/hudi-common/src/test/resources/log4j-surefire-quiet.properties @@ -17,6 +17,7 @@ ### log4j.rootLogger=WARN, CONSOLE log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR # CONSOLE is set to be a ConsoleAppender. log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender diff --git a/hudi-common/src/test/resources/log4j-surefire.properties b/hudi-common/src/test/resources/log4j-surefire.properties index c03e808cca1f8..c5bdf75ae2ae3 100644 --- a/hudi-common/src/test/resources/log4j-surefire.properties +++ b/hudi-common/src/test/resources/log4j-surefire.properties @@ -18,6 +18,7 @@ log4j.rootLogger=WARN, CONSOLE log4j.logger.org.apache=INFO log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR # A1 is set to be a ConsoleAppender. log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender diff --git a/hudi-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/hive/HoodieCombineHiveInputFormat.java b/hudi-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/hive/HoodieCombineHiveInputFormat.java index 2220a10d0ff05..02afd19fe86f4 100644 --- a/hudi-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/hive/HoodieCombineHiveInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/hive/HoodieCombineHiveInputFormat.java @@ -39,8 +39,7 @@ protected String getParquetRealtimeInputFormatClassName() { } @Override - protected org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim - createInputFormatShim() { + protected org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim createInputFormatShim() { return new HoodieCombineFileInputFormatShim<>(); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java new file mode 100644 index 0000000000000..437304fb043d0 --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop; + +import org.apache.hadoop.mapred.FileSplit; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + + +/** + * Sub-type of File Split which encapsulates both skeleton and bootstrap base file splits. + */ +public class BootstrapBaseFileSplit extends FileSplit { + + private FileSplit bootstrapFileSplit; + + public BootstrapBaseFileSplit() { + super(); + } + + public BootstrapBaseFileSplit(FileSplit baseSplit, FileSplit bootstrapFileSplit) + throws IOException { + super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(), baseSplit.getLocations()); + this.bootstrapFileSplit = bootstrapFileSplit; + } + + public FileSplit getBootstrapFileSplit() { + return bootstrapFileSplit; + } + + @Override + public void write(DataOutput out) throws IOException { + super.write(out); + bootstrapFileSplit.write(out); + } + + @Override + public void readFields(DataInput in) throws IOException { + super.readFields(in); + bootstrapFileSplit = new WrapperFileSplit(); + bootstrapFileSplit.readFields(in); + } + + /** + * Wrapper for FileSplit just to expose default constructor to the outer class. + */ + public static class WrapperFileSplit extends FileSplit { + + public WrapperFileSplit() { + super(); + } + } +} \ No newline at end of file diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapColumnStichingRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapColumnStichingRecordReader.java new file mode 100644 index 0000000000000..d3a049ae952e3 --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapColumnStichingRecordReader.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop; + +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordReader; + +import java.io.IOException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Stitches 2 record reader returned rows and presents a concatenated view to clients. + */ +public class BootstrapColumnStichingRecordReader implements RecordReader { + + private static final Logger LOG = LogManager.getLogger(BootstrapColumnStichingRecordReader.class); + + private final RecordReader leftColsRecordReader; + private final RecordReader rightColsRecordReader; + private final int numLeftColumns; + private final ArrayWritable values; + private final boolean validate; + + public BootstrapColumnStichingRecordReader(RecordReader left, + int numLeftColumns, RecordReader right, int numRightColumns, boolean validate) { + this.leftColsRecordReader = left; + this.rightColsRecordReader = right; + this.validate = validate; + this.numLeftColumns = numLeftColumns; + + ArrayWritable rightW = rightColsRecordReader.createValue(); + int numColumns = numLeftColumns + numRightColumns; + if (rightW.getValueClass() != null) { + values = new ArrayWritable(rightW.getValueClass(), new Writable[numColumns]); + } else { + String[] vals = IntStream.range(0, numColumns).mapToObj(idx -> "").collect(Collectors.toList()) + .toArray(new String[0]); + values = new ArrayWritable(vals); + } + LOG.info("Total ArrayWritable Length :" + values.get().length); + } + + @Override + public boolean next(NullWritable key, ArrayWritable value) throws IOException { + ArrayWritable left = leftColsRecordReader.createValue(); + ArrayWritable right = rightColsRecordReader.createValue(); + + boolean hasMoreOnLeft = leftColsRecordReader.next(leftColsRecordReader.createKey(), left); + boolean hasMoreOnRight = rightColsRecordReader.next(rightColsRecordReader.createKey(), right); + if (validate) { + ValidationUtils.checkArgument(hasMoreOnLeft == hasMoreOnRight, + String.format("hasMoreOnLeft:%s, hasMoreOnRight: %s", hasMoreOnLeft, hasMoreOnRight)); + } + for (int i = 0; i < numLeftColumns; i++) { + value.get()[i] = left.get()[i]; + } + for (int j = numLeftColumns; j < right.get().length; j++) { + value.get()[j] = right.get()[j]; + } + return hasMoreOnLeft && hasMoreOnRight; + } + + @Override + public NullWritable createKey() { + return leftColsRecordReader.createKey(); + } + + @Override + public ArrayWritable createValue() { + return values; + } + + @Override + public long getPos() throws IOException { + return leftColsRecordReader.getPos(); + } + + @Override + public void close() throws IOException { + leftColsRecordReader.close(); + rightColsRecordReader.close(); + } + + @Override + public float getProgress() throws IOException { + return leftColsRecordReader.getProgress(); + } +} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/FileStatusWithBootstrapBaseFile.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/FileStatusWithBootstrapBaseFile.java new file mode 100644 index 0000000000000..11345cfeba88e --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/FileStatusWithBootstrapBaseFile.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; + +/** + * Sub-Type of File Status tracking both skeleton and bootstrap base file's status. + */ +public class FileStatusWithBootstrapBaseFile extends FileStatus { + + private final FileStatus bootstrapBaseFileStatus; + + public FileStatusWithBootstrapBaseFile(FileStatus fileStatus, FileStatus bootstrapBaseFileStatus) throws IOException { + super(fileStatus); + this.bootstrapBaseFileStatus = bootstrapBaseFileStatus; + } + + @Override + public Path getPath() { + return new PathWithBootstrapFileStatus(super.getPath(), bootstrapBaseFileStatus); + } +} \ No newline at end of file diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieColumnProjectionUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieColumnProjectionUtils.java new file mode 100644 index 0000000000000..b7141a8ee762f --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieColumnProjectionUtils.java @@ -0,0 +1,300 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop; + +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.io.IOConstants; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Utility funcitons copied from Hive ColumnProjectionUtils.java. + * Needed to copy as we see NoSuchMethod errors when directly using these APIs with/without Spark. + * Some of these methods are not available across hive versions. + */ +public class HoodieColumnProjectionUtils { + public static final Logger LOG = LoggerFactory.getLogger(ColumnProjectionUtils.class); + + public static final String READ_COLUMN_IDS_CONF_STR = "hive.io.file.readcolumn.ids"; + /** + * the nested column path is the string from the root to the leaf + * e.g. + * c:struct_of (a:string,b:string). + * the column a's path is c.a and b's path is c.b + */ + public static final String READ_NESTED_COLUMN_PATH_CONF_STR = + "hive.io.file.readNestedColumn.paths"; + public static final String READ_ALL_COLUMNS = "hive.io.file.read.all.columns"; + public static final String READ_COLUMN_NAMES_CONF_STR = "hive.io.file.readcolumn.names"; + private static final String READ_COLUMN_IDS_CONF_STR_DEFAULT = ""; + private static final String READ_COLUMN_NAMES_CONF_STR_DEFAULT = ""; + private static final String READ_NESTED_COLUMN_PATH_CONF_STR_DEFAULT = ""; + private static final boolean READ_ALL_COLUMNS_DEFAULT = true; + + private static final String COMMA = ","; + + /** Special Column Names added during Parquet Projection. **/ + public static final String PARQUET_BLOCK_OFFSET_COL_NAME = "BLOCK__OFFSET__INSIDE__FILE"; + public static final String PARQUET_INPUT_FILE_NAME = "INPUT__FILE__NAME"; + public static final String PARQUET_ROW_ID = "ROW__ID"; + + public static final List PARQUET_SPECIAL_COLUMN_NAMES = CollectionUtils + .createImmutableList(PARQUET_BLOCK_OFFSET_COL_NAME, PARQUET_INPUT_FILE_NAME, + PARQUET_ROW_ID); + + /** + * Sets the READ_ALL_COLUMNS flag and removes any previously + * set column ids. + */ + public static void setReadAllColumns(Configuration conf) { + conf.setBoolean(READ_ALL_COLUMNS, true); + setReadColumnIDConf(conf, READ_COLUMN_IDS_CONF_STR_DEFAULT); + setReadColumnNamesConf(conf, READ_COLUMN_NAMES_CONF_STR_DEFAULT); + } + + /** + * Returns the READ_ALL_COLUMNS columns flag. + */ + public static boolean isReadAllColumns(Configuration conf) { + return conf.getBoolean(READ_ALL_COLUMNS, READ_ALL_COLUMNS_DEFAULT); + } + + /** + * Sets the READ_ALL_COLUMNS flag to false and overwrites column ids + * with the provided list. + */ + public static void setReadColumns(Configuration conf, List ids, List names) { + setReadColumnIDConf(conf, READ_COLUMN_IDS_CONF_STR_DEFAULT); + setReadColumnNamesConf(conf, READ_COLUMN_NAMES_CONF_STR_DEFAULT); + appendReadColumns(conf, ids); + appendReadColumnNames(conf, names); + } + + /** + * Appends read columns' ids (start from zero). Once a column + * is included in the list, a underlying record reader of a columnar file format + * (e.g. RCFile and ORC) can know what columns are needed. + */ + public static void appendReadColumns(Configuration conf, List ids) { + String id = toReadColumnIDString(ids); + String old = conf.get(READ_COLUMN_IDS_CONF_STR, null); + String newConfStr = id; + if (old != null && !old.isEmpty()) { + newConfStr = newConfStr + StringUtils.COMMA_STR + old; + } + setReadColumnIDConf(conf, newConfStr); + // Set READ_ALL_COLUMNS to false + conf.setBoolean(READ_ALL_COLUMNS, false); + } + + /** + * Appends read nested column's paths. Once a read nested column path + * is included in the list, a underlying record reader of a columnar file format + * (e.g. Parquet and ORC) can know what columns are needed. + */ + public static void appendNestedColumnPaths( + Configuration conf, + List paths) { + if (paths == null || paths.isEmpty()) { + return; + } + String pathsStr = StringUtils.join(StringUtils.COMMA_STR, + paths.toArray(new String[paths.size()])); + String old = conf.get(READ_NESTED_COLUMN_PATH_CONF_STR, null); + String newConfStr = pathsStr; + if (old != null && !old.isEmpty()) { + newConfStr = newConfStr + StringUtils.COMMA_STR + old; + } + setReadNestedColumnPathConf(conf, newConfStr); + } + + + /** + * This method appends read column information to configuration to use for PPD. It is + * currently called with information from TSOP. Names come from TSOP input RowSchema, and + * IDs are the indexes inside the schema (which PPD assumes correspond to indexes inside the + * files to PPD in; something that would be invalid in many cases of schema evolution). + * @param conf Config to set values to. + * @param ids Column ids. + * @param names Column names. + */ + public static void appendReadColumns( + Configuration conf, List ids, List names, List groupPaths) { + if (ids.size() != names.size()) { + LOG.warn("Read column counts do not match: " + + ids.size() + " ids, " + names.size() + " names"); + } + appendReadColumns(conf, ids); + appendReadColumnNames(conf, names); + appendNestedColumnPaths(conf, groupPaths); + } + + public static void appendReadColumns( + StringBuilder readColumnsBuffer, StringBuilder readColumnNamesBuffer, List ids, + List names) { + String preppedIdStr = ids.stream().map(x -> String.valueOf(x)).collect(Collectors.joining(COMMA)); + String preppedNamesStr = names.stream().collect(Collectors.joining(COMMA)); + if (readColumnsBuffer.length() > 0) { + readColumnsBuffer.append(COMMA); + } + readColumnsBuffer.append(preppedIdStr); + if (readColumnNamesBuffer.length() > 0) { + readColumnNamesBuffer.append(COMMA); + } + readColumnNamesBuffer.append(preppedNamesStr); + } + + /** + * Returns an array of column ids(start from zero) which is set in the given + * parameter conf. + */ + public static List getReadColumnIDs(Configuration conf) { + String skips = conf.get(READ_COLUMN_IDS_CONF_STR, READ_COLUMN_IDS_CONF_STR_DEFAULT); + String[] list = StringUtils.split(skips); + List result = new ArrayList(list.length); + for (String element : list) { + // it may contain duplicates, remove duplicates + Integer toAdd = Integer.parseInt(element); + if (!result.contains(toAdd)) { + result.add(toAdd); + } + // NOTE: some code uses this list to correlate with column names, and yet these lists may + // contain duplicates, which this call will remove and the other won't. As far as I can + // tell, no code will actually use these two methods together; all is good if the code + // gets the ID list without relying on this method. Or maybe it just works by magic. + } + return result; + } + + public static Set getNestedColumnPaths(Configuration conf) { + String skips = + conf.get(READ_NESTED_COLUMN_PATH_CONF_STR, READ_NESTED_COLUMN_PATH_CONF_STR_DEFAULT); + return new HashSet<>(Arrays.asList(StringUtils.split(skips))); + } + + public static String[] getReadColumnNames(Configuration conf) { + String colNames = conf.get(READ_COLUMN_NAMES_CONF_STR, READ_COLUMN_NAMES_CONF_STR_DEFAULT); + if (colNames != null && !colNames.isEmpty()) { + return colNames.split(","); + } + return new String[] {}; + } + + public static List getIOColumns(Configuration conf) { + String colNames = conf.get(IOConstants.COLUMNS, ""); + if (colNames != null && !colNames.isEmpty()) { + return Arrays.asList(colNames.split(",")); + } + return new ArrayList<>(); + } + + public static List getIOColumnTypes(Configuration conf) { + String colTypes = conf.get(IOConstants.COLUMNS_TYPES, ""); + if (colTypes != null && !colTypes.isEmpty()) { + return TypeInfoUtils.getTypeInfosFromTypeString(colTypes).stream() + .map(t -> t.getTypeName()).collect(Collectors.toList()); + } + return new ArrayList<>(); + } + + public static List> getIOColumnNameAndTypes(Configuration conf) { + List names = getIOColumns(conf); + List types = getIOColumnTypes(conf); + ValidationUtils.checkArgument(names.size() == types.size()); + return IntStream.range(0, names.size()).mapToObj(idx -> Pair.of(names.get(idx), types.get(idx))) + .collect(Collectors.toList()); + } + + public static void setIOColumnNameAndTypes(Configuration conf, List> colNamesAndTypes) { + String colNames = colNamesAndTypes.stream().map(e -> e.getKey()).collect(Collectors.joining(",")); + String colTypes = colNamesAndTypes.stream().map(e -> e.getValue()).collect(Collectors.joining(",")); + conf.set(IOConstants.COLUMNS, colNames); + conf.set(IOConstants.COLUMNS_TYPES, colTypes); + } + + private static void setReadColumnIDConf(Configuration conf, String id) { + if (id.trim().isEmpty()) { + conf.set(READ_COLUMN_IDS_CONF_STR, READ_COLUMN_IDS_CONF_STR_DEFAULT); + } else { + conf.set(READ_COLUMN_IDS_CONF_STR, id); + } + } + + private static void setReadColumnNamesConf(Configuration conf, String id) { + if (id.trim().isEmpty()) { + conf.set(READ_COLUMN_NAMES_CONF_STR, READ_COLUMN_NAMES_CONF_STR_DEFAULT); + } else { + conf.set(READ_COLUMN_NAMES_CONF_STR, id); + } + } + + private static void setReadNestedColumnPathConf( + Configuration conf, + String nestedColumnPaths) { + nestedColumnPaths = nestedColumnPaths.toLowerCase(); + if (nestedColumnPaths.trim().isEmpty()) { + conf.set(READ_NESTED_COLUMN_PATH_CONF_STR, READ_NESTED_COLUMN_PATH_CONF_STR_DEFAULT); + } else { + conf.set(READ_NESTED_COLUMN_PATH_CONF_STR, nestedColumnPaths); + } + } + + private static void appendReadColumnNames(Configuration conf, List cols) { + String old = conf.get(READ_COLUMN_NAMES_CONF_STR, ""); + StringBuilder result = new StringBuilder(old); + boolean first = old.isEmpty(); + for (String col: cols) { + if (first) { + first = false; + } else { + result.append(','); + } + result.append(col); + } + conf.set(READ_COLUMN_NAMES_CONF_STR, result.toString()); + } + + private static String toReadColumnIDString(List ids) { + String id = ""; + for (int i = 0; i < ids.size(); i++) { + if (i == 0) { + id = id + ids.get(i); + } else { + id = id + StringUtils.COMMA_STR + ids.get(i); + } + } + return id; + } + +} \ No newline at end of file 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 266cc6f874be3..74a557e7efc1d 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 @@ -19,21 +19,29 @@ package org.apache.hudi.hadoop; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; +import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg; +import org.apache.hadoop.hive.ql.metadata.VirtualColumn; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; @@ -47,6 +55,8 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; /** * HoodieInputFormat which understands the Hoodie File Structure and filters files based on the Hoodie Mode. If paths @@ -115,6 +125,8 @@ public FileStatus[] listStatus(JobConf job) throws IOException { return returns.toArray(new FileStatus[returns.size()]); } + + /** * Achieves listStatus functionality for an incrementally queried table. Instead of listing all * partitions and then filtering based on the commits of interest, this logic first extracts the @@ -153,7 +165,7 @@ public Configuration getConf() { @Override public RecordReader getRecordReader(final InputSplit split, final JobConf job, - final Reporter reporter) throws IOException { + final Reporter reporter) throws IOException { // TODO enable automatic predicate pushdown after fixing issues // FileSplit fileSplit = (FileSplit) split; // HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent()); @@ -166,7 +178,91 @@ public RecordReader getRecordReader(final InputSpli // ParquetInputFormat.setFilterPredicate(job, predicate); // clearOutExistingPredicate(job); // } + if (split instanceof BootstrapBaseFileSplit) { + BootstrapBaseFileSplit eSplit = (BootstrapBaseFileSplit)split; + String[] rawColNames = HoodieColumnProjectionUtils.getReadColumnNames(job); + List rawColIds = HoodieColumnProjectionUtils.getReadColumnIDs(job); + List> projectedColsWithIndex = + IntStream.range(0, rawColIds.size()).mapToObj(idx -> Pair.of(rawColIds.get(idx), rawColNames[idx])) + .collect(Collectors.toList()); + + List> hoodieColsProjected = projectedColsWithIndex.stream() + .filter(idxWithName -> HoodieRecord.HOODIE_META_COLUMNS.contains(idxWithName.getValue())) + .collect(Collectors.toList()); + List> externalColsProjected = projectedColsWithIndex.stream() + .filter(idxWithName -> !HoodieRecord.HOODIE_META_COLUMNS.contains(idxWithName.getValue()) + && !VirtualColumn.VIRTUAL_COLUMN_NAMES.contains(idxWithName.getValue())) + .collect(Collectors.toList()); + + // This always matches hive table description + List> colNameWithTypes = HoodieColumnProjectionUtils.getIOColumnNameAndTypes(job); + List> colNamesWithTypesForExternal = colNameWithTypes.stream() + .filter(p -> !HoodieRecord.HOODIE_META_COLUMNS.contains(p.getKey())).collect(Collectors.toList()); + LOG.info("colNameWithTypes =" + colNameWithTypes + ", Num Entries =" + colNameWithTypes.size()); + if (hoodieColsProjected.isEmpty()) { + return super.getRecordReader(eSplit.getBootstrapFileSplit(), job, reporter); + } else if (externalColsProjected.isEmpty()) { + return super.getRecordReader(split, job, reporter); + } else { + FileSplit rightSplit = eSplit.getBootstrapFileSplit(); + // Hive PPD works at row-group level and only enabled when hive.optimize.index.filter=true; + // The above config is disabled by default. But when enabled, would cause misalignment between + // skeleton and bootstrap file. We will disable them specifically when query needs bootstrap and skeleton + // file to be stitched. + // This disables row-group filtering + JobConf jobConfCopy = new JobConf(job); + jobConfCopy.unset(TableScanDesc.FILTER_EXPR_CONF_STR); + jobConfCopy.unset(ConvertAstToSearchArg.SARG_PUSHDOWN); + + LOG.info("Generating column stitching reader for " + eSplit.getPath() + " and " + rightSplit.getPath()); + return new BootstrapColumnStichingRecordReader(super.getRecordReader(eSplit, jobConfCopy, reporter), + HoodieRecord.HOODIE_META_COLUMNS.size(), + super.getRecordReader(rightSplit, jobConfCopy, reporter), + colNamesWithTypesForExternal.size(), + true); + } + } + if (LOG.isDebugEnabled()) { + LOG.debug("EMPLOYING DEFAULT RECORD READER - " + split); + } return super.getRecordReader(split, job, reporter); } -} + @Override + protected boolean isSplitable(FileSystem fs, Path filename) { + return !(filename instanceof PathWithBootstrapFileStatus); + } + + @Override + protected FileSplit makeSplit(Path file, long start, long length, + String[] hosts) { + FileSplit split = new FileSplit(file, start, length, hosts); + + if (file instanceof PathWithBootstrapFileStatus) { + return makeExternalFileSplit((PathWithBootstrapFileStatus)file, split); + } + return split; + } + + @Override + protected FileSplit makeSplit(Path file, long start, long length, + String[] hosts, String[] inMemoryHosts) { + FileSplit split = new FileSplit(file, start, length, hosts, inMemoryHosts); + if (file instanceof PathWithBootstrapFileStatus) { + return makeExternalFileSplit((PathWithBootstrapFileStatus)file, split); + } + return split; + } + + private BootstrapBaseFileSplit makeExternalFileSplit(PathWithBootstrapFileStatus file, FileSplit split) { + try { + LOG.info("Making external data split for " + file); + FileStatus externalFileStatus = file.getBootstrapFileStatus(); + FileSplit externalFileSplit = makeSplit(externalFileStatus.getPath(), 0, externalFileStatus.getLen(), + new String[0], new String[0]); + return new BootstrapBaseFileSplit(split, externalFileSplit); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } +} \ No newline at end of file diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputSplitUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputSplitUtils.java new file mode 100644 index 0000000000000..745657183d7bd --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputSplitUtils.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; + +public class InputSplitUtils { + + public static void writeString(String str, DataOutput out) throws IOException { + byte[] bytes = str.getBytes(StandardCharsets.UTF_8); + out.writeInt(bytes.length); + out.write(bytes); + } + + public static String readString(DataInput in) throws IOException { + byte[] bytes = new byte[in.readInt()]; + in.readFully(bytes); + return new String(bytes, StandardCharsets.UTF_8); + } + + /** + * Return correct base-file schema based on split. + * + * @param split File Split + * @param conf Configuration + * @return + */ + public static Schema getBaseFileSchema(FileSplit split, Configuration conf) { + try { + if (split instanceof BootstrapBaseFileSplit) { + HoodieFileReader storageReader = HoodieFileReaderFactory.getFileReader(conf, + ((BootstrapBaseFileSplit)(split)).getBootstrapFileSplit().getPath()); + return HoodieAvroUtils.addMetadataFields(storageReader.getSchema()); + } + return HoodieRealtimeRecordReaderUtils.readSchema(conf, split.getPath()); + } catch (IOException e) { + throw new HoodieIOException("Failed to read footer for parquet " + split.getPath(), e); + } + } +} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/LocatedFileStatusWithBootstrapBaseFile.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/LocatedFileStatusWithBootstrapBaseFile.java new file mode 100644 index 0000000000000..935272dc7cfa3 --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/LocatedFileStatusWithBootstrapBaseFile.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; + +/** + * Sub-Type of File Status tracking both skeleton and bootstrap base file's status. + */ +public class LocatedFileStatusWithBootstrapBaseFile extends LocatedFileStatus { + + private final FileStatus bootstrapFileStatus; + + public LocatedFileStatusWithBootstrapBaseFile(LocatedFileStatus fileStatus, FileStatus bootstrapFileStatus) + throws IOException { + super(fileStatus, fileStatus.getBlockLocations()); + this.bootstrapFileStatus = bootstrapFileStatus; + } + + @Override + public Path getPath() { + return new PathWithBootstrapFileStatus(super.getPath(), bootstrapFileStatus); + } +} \ No newline at end of file diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithBootstrapFileStatus.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithBootstrapFileStatus.java new file mode 100644 index 0000000000000..206217f4fd79a --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithBootstrapFileStatus.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +/** + * Hacky Workaround !!! + * With the base input format implementations in Hadoop/Hive, + * we need to encode additional information in Path to track matching external file. + * Hence, this weird looking class which tracks an external file status + * in Path. + */ +public class PathWithBootstrapFileStatus extends Path { + + private final FileStatus bootstrapFileStatus; + + public PathWithBootstrapFileStatus(Path path, FileStatus bootstrapFileStatus) { + super(path.getParent(), path.getName()); + this.bootstrapFileStatus = bootstrapFileStatus; + } + + public FileStatus getBootstrapFileStatus() { + return bootstrapFileStatus; + } +} \ No newline at end of file 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 e14fe7ed2b4c7..65c416c8bc72c 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 @@ -22,6 +22,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.LogReaderUtils; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.InputSplitUtils; import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; @@ -29,6 +30,7 @@ import org.apache.avro.Schema.Field; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -46,7 +48,7 @@ public abstract class AbstractRealtimeRecordReader { private static final Logger LOG = LogManager.getLogger(AbstractRealtimeRecordReader.class); - protected final HoodieRealtimeFileSplit split; + protected final RealtimeSplit split; protected final JobConf jobConf; protected final boolean usesCustomPayload; // Schema handles @@ -54,7 +56,7 @@ public abstract class AbstractRealtimeRecordReader { private Schema writerSchema; private Schema hiveSchema; - public AbstractRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job) { + public AbstractRealtimeRecordReader(RealtimeSplit split, JobConf job) { this.split = split; this.jobConf = job; LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); @@ -81,14 +83,13 @@ private boolean usesCustomPayload() { * job conf. */ private void init() throws IOException { - Schema schemaFromLogFile = - LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaLogPaths(), jobConf); + Schema schemaFromLogFile = LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaLogPaths(), jobConf); if (schemaFromLogFile == null) { - writerSchema = HoodieRealtimeRecordReaderUtils.readSchema(jobConf, split.getPath()); - LOG.debug("Writer Schema From Parquet => " + writerSchema.getFields()); + writerSchema = InputSplitUtils.getBaseFileSchema((FileSplit)split, jobConf); + LOG.info("Writer Schema From Parquet => " + writerSchema.getFields()); } else { writerSchema = schemaFromLogFile; - LOG.debug("Writer Schema From Log => " + writerSchema.getFields()); + LOG.info("Writer Schema From Log => " + writerSchema.toString(true)); } // Add partitioning fields to writer schema for resulting row to contain null values for these fields String partitionFields = jobConf.get(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, ""); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java index 11247910effaf..9cb0ada857b77 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.hadoop.HoodieColumnProjectionUtils; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.UseFileSplitsFromInputFormat; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; @@ -42,7 +43,9 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.stream.Stream; /** @@ -110,10 +113,20 @@ private static Configuration addProjectionField(Configuration conf, String field } private static void addRequiredProjectionFields(Configuration configuration) { - // Need this to do merge records in HoodieRealtimeRecordReader - addProjectionField(configuration, HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS); - addProjectionField(configuration, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieInputFormatUtils.HOODIE_COMMIT_TIME_COL_POS); - addProjectionField(configuration, HoodieRecord.PARTITION_PATH_METADATA_FIELD, HoodieInputFormatUtils.HOODIE_PARTITION_PATH_COL_POS); + List projectedIds = new ArrayList<>(HoodieColumnProjectionUtils.getReadColumnIDs(configuration)); + List projectedNames = new ArrayList<>(Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(configuration))); + projectedIds.addAll(Arrays.asList( + HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS, + HoodieInputFormatUtils.HOODIE_COMMIT_TIME_COL_POS, + HoodieInputFormatUtils.HOODIE_PARTITION_PATH_COL_POS) + ); + projectedNames.addAll(Arrays.asList( + HoodieRecord.RECORD_KEY_METADATA_FIELD, + HoodieRecord.COMMIT_TIME_METADATA_FIELD, + HoodieRecord.PARTITION_PATH_METADATA_FIELD) + ); + + HoodieColumnProjectionUtils.setReadColumns(configuration, projectedIds, projectedNames); } /** @@ -134,7 +147,7 @@ private static void cleanProjectionColumnIds(Configuration conf) { @Override public RecordReader getRecordReader(final InputSplit split, final JobConf jobConf, - final Reporter reporter) throws IOException { + final Reporter reporter) throws IOException { // Hive on Spark invokes multiple getRecordReaders from different threads in the same spark task (and hence the // same JVM) unlike Hive on MR. Due to this, accesses to JobConf, which is shared across all threads, is at the // risk of experiencing race conditions. Hence, we synchronize on the JobConf object here. There is negligible @@ -164,11 +177,12 @@ public RecordReader getRecordReader(final InputSpli LOG.info("Creating record reader with readCols :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR) + ", Ids :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)); + // sanity check - ValidationUtils.checkArgument(split instanceof HoodieRealtimeFileSplit, - "HoodieRealtimeRecordReader can only work on HoodieRealtimeFileSplit and not with " + split); + ValidationUtils.checkArgument(split instanceof RealtimeSplit, + "HoodieRealtimeRecordReader can only work on RealtimeSplit and not with " + split); - return new HoodieRealtimeRecordReader((HoodieRealtimeFileSplit) split, jobConf, + return new HoodieRealtimeRecordReader((RealtimeSplit) split, jobConf, super.getRecordReader(split, jobConf, reporter)); } @@ -176,4 +190,4 @@ public RecordReader getRecordReader(final InputSpli public Configuration getConf() { return conf; } -} \ No newline at end of file +} 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 908ec8778aa4e..fe481f0a202dc 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 @@ -23,14 +23,12 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; import java.util.List; /** * Filesplit that wraps the base split and a list of log files to merge deltas from. */ -public class HoodieRealtimeFileSplit extends FileSplit { +public class HoodieRealtimeFileSplit extends FileSplit implements RealtimeSplit { private List deltaLogPaths; @@ -62,39 +60,28 @@ public String getBasePath() { return basePath; } - private static void writeString(String str, DataOutput out) throws IOException { - byte[] bytes = str.getBytes(StandardCharsets.UTF_8); - out.writeInt(bytes.length); - out.write(bytes); + public void setDeltaLogPaths(List deltaLogPaths) { + this.deltaLogPaths = deltaLogPaths; } - private static String readString(DataInput in) throws IOException { - byte[] bytes = new byte[in.readInt()]; - in.readFully(bytes); - return new String(bytes, StandardCharsets.UTF_8); + public void setMaxCommitTime(String maxCommitTime) { + this.maxCommitTime = maxCommitTime; + } + + public void setBasePath(String basePath) { + this.basePath = basePath; } @Override public void write(DataOutput out) throws IOException { super.write(out); - writeString(basePath, out); - writeString(maxCommitTime, out); - out.writeInt(deltaLogPaths.size()); - for (String logFilePath : deltaLogPaths) { - writeString(logFilePath, out); - } + writeToOutput(out); } @Override public void readFields(DataInput in) throws IOException { super.readFields(in); - basePath = readString(in); - maxCommitTime = readString(in); - int totalLogFiles = in.readInt(); - deltaLogPaths = new ArrayList<>(totalLogFiles); - for (int i = 0; i < totalLogFiles; i++) { - deltaLogPaths.add(readString(in)); - } + readFromInput(in); } @Override diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java index a8af067e5bf1a..1e3a25ac78a11 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -42,7 +42,7 @@ public class HoodieRealtimeRecordReader implements RecordReader reader; - public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job, + public HoodieRealtimeRecordReader(RealtimeSplit split, JobConf job, RecordReader realReader) { this.reader = constructRecordReader(split, job, realReader); } @@ -59,7 +59,7 @@ public static boolean canSkipMerging(JobConf jobConf) { * @param realReader Parquet Record Reader * @return Realtime Reader */ - private static RecordReader constructRecordReader(HoodieRealtimeFileSplit split, + private static RecordReader constructRecordReader(RealtimeSplit split, JobConf jobConf, RecordReader realReader) { try { if (canSkipMerging(jobConf)) { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java new file mode 100644 index 0000000000000..fd3b5b81074aa --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop.realtime; + +import org.apache.hudi.hadoop.BootstrapBaseFileSplit; + +import org.apache.hadoop.mapred.FileSplit; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.List; + +/** + * Realtime File Split with external base file. + */ +public class RealtimeBootstrapBaseFileSplit extends BootstrapBaseFileSplit implements RealtimeSplit { + + private List deltaLogPaths; + + private String maxInstantTime; + + private String basePath; + + public RealtimeBootstrapBaseFileSplit() { + super(); + } + + public RealtimeBootstrapBaseFileSplit(FileSplit baseSplit, String basePath, List deltaLogPaths, + String maxInstantTime, FileSplit externalFileSplit) throws IOException { + super(baseSplit, externalFileSplit); + this.maxInstantTime = maxInstantTime; + this.deltaLogPaths = deltaLogPaths; + this.basePath = basePath; + } + + @Override + public void write(DataOutput out) throws IOException { + super.write(out); + writeToOutput(out); + } + + @Override + public void readFields(DataInput in) throws IOException { + super.readFields(in); + readFromInput(in); + } + + @Override + public List getDeltaLogPaths() { + return deltaLogPaths; + } + + @Override + public String getMaxCommitTime() { + return maxInstantTime; + } + + @Override + public String getBasePath() { + return basePath; + } + + @Override + public void setDeltaLogPaths(List deltaLogPaths) { + this.deltaLogPaths = deltaLogPaths; + } + + @Override + public void setMaxCommitTime(String maxInstantTime) { + this.maxInstantTime = maxInstantTime; + } + + @Override + public void setBasePath(String basePath) { + this.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 02bb5eb63c130..78925c3581404 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 @@ -48,7 +48,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader protected final RecordReader parquetReader; private final Map> deltaRecordMap; - public RealtimeCompactedRecordReader(HoodieRealtimeFileSplit split, JobConf job, + public RealtimeCompactedRecordReader(RealtimeSplit split, JobConf job, RecordReader realReader) throws IOException { super(split, job); this.parquetReader = realReader; @@ -120,7 +120,10 @@ public boolean next(NullWritable aVoid, ArrayWritable arrayWritable) throws IOEx } Writable[] originalValue = arrayWritable.get(); try { - System.arraycopy(replaceValue, 0, originalValue, 0, originalValue.length); + // Sometime originalValue.length > replaceValue.length. + // This can happen when hive query is looking for pseudo parquet columns like BLOCK_OFFSET_INSIDE_FILE + System.arraycopy(replaceValue, 0, originalValue, 0, + Math.min(originalValue.length, replaceValue.length)); arrayWritable.set(originalValue); } catch (RuntimeException re) { LOG.error("Got exception when doing array copy", re); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java new file mode 100644 index 0000000000000..0fab734342e49 --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop.realtime; + +import org.apache.hudi.hadoop.InputSplitUtils; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.InputSplitWithLocationInfo; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Realtime Input Split Interface. + */ +public interface RealtimeSplit extends InputSplitWithLocationInfo { + + /** + * Return Log File Paths. + * @return + */ + List getDeltaLogPaths(); + + /** + * Return Max Instant Time. + * @return + */ + String getMaxCommitTime(); + + /** + * Return Base Path of the dataset. + * @return + */ + String getBasePath(); + + /** + * Update Log File Paths. + * @param deltaLogPaths + */ + void setDeltaLogPaths(List deltaLogPaths); + + /** + * Update Maximum valid instant time. + * @param maxCommitTime + */ + void setMaxCommitTime(String maxCommitTime); + + /** + * Set Base Path. + * @param basePath + */ + void setBasePath(String basePath); + + default void writeToOutput(DataOutput out) throws IOException { + InputSplitUtils.writeString(getBasePath(), out); + InputSplitUtils.writeString(getMaxCommitTime(), out); + out.writeInt(getDeltaLogPaths().size()); + for (String logFilePath : getDeltaLogPaths()) { + InputSplitUtils.writeString(logFilePath, out); + } + } + + default void readFromInput(DataInput in) throws IOException { + setBasePath(InputSplitUtils.readString(in)); + setMaxCommitTime(InputSplitUtils.readString(in)); + int totalLogFiles = in.readInt(); + List deltaLogPaths = new ArrayList<>(totalLogFiles); + for (int i = 0; i < totalLogFiles; i++) { + deltaLogPaths.add(InputSplitUtils.readString(in)); + } + setDeltaLogPaths(deltaLogPaths); + } + + /** + * The file containing this split's data. + */ + public Path getPath(); + + /** + * The position of the first byte in the file to process. + */ + public long getStart(); + + /** + * The number of bytes in the file to process. + */ + public long getLength(); +} 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 8bc1cfb41224b..c06bff26e8f93 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 @@ -66,7 +66,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader * @param job Job Configuration * @param realReader Parquet Reader */ - public RealtimeUnmergedRecordReader(HoodieRealtimeFileSplit split, JobConf job, + public RealtimeUnmergedRecordReader(RealtimeSplit split, JobConf job, RecordReader realReader) { super(split, job); this.parquetReader = new SafeParquetRecordReaderWrapper(realReader); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index 6d967f3260321..bf46f46fb6965 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -32,11 +32,14 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.FileStatusWithBootstrapBaseFile; import org.apache.hudi.hadoop.HoodieParquetInputFormat; +import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile; import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat; import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; @@ -108,7 +111,7 @@ public static String getSerDeClassName(HoodieFileFormat baseFileFormat) { } public static FileInputFormat getInputFormat(String path, boolean realtime, Configuration conf) { - final String extension = FSUtils.getFileExtension(path.toString()); + final String extension = FSUtils.getFileExtension(path); if (extension.equals(HoodieFileFormat.PARQUET.getFileExtension())) { return getInputFormat(HoodieFileFormat.PARQUET, realtime, conf); } @@ -281,6 +284,19 @@ public static HoodieTableMetaClient getTableMetaClientForBasePath(FileSystem fs, return new HoodieTableMetaClient(fs.getConf(), baseDir.toString()); } + public static FileStatus getFileStatus(HoodieBaseFile baseFile) throws IOException { + if (baseFile.getBootstrapBaseFile().isPresent()) { + if (baseFile.getFileStatus() instanceof LocatedFileStatus) { + return new LocatedFileStatusWithBootstrapBaseFile((LocatedFileStatus)baseFile.getFileStatus(), + baseFile.getBootstrapBaseFile().get().getFileStatus()); + } else { + return new FileStatusWithBootstrapBaseFile(baseFile.getFileStatus(), + baseFile.getBootstrapBaseFile().get().getFileStatus()); + } + } + return baseFile.getFileStatus(); + } + /** * Filter a list of FileStatus based on commitsToCheck for incremental view. * @param job @@ -291,7 +307,7 @@ public static HoodieTableMetaClient getTableMetaClientForBasePath(FileSystem fs, * @return */ public static List filterIncrementalFileStatus(Job job, HoodieTableMetaClient tableMetaClient, - HoodieTimeline timeline, FileStatus[] fileStatuses, List commitsToCheck) { + HoodieTimeline timeline, FileStatus[] fileStatuses, List commitsToCheck) throws IOException { TableFileSystemView.BaseFileOnlyView roView = new HoodieTableFileSystemView(tableMetaClient, timeline, fileStatuses); List commitsList = commitsToCheck.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); List filteredFiles = roView.getLatestBaseFilesInRange(commitsList).collect(Collectors.toList()); @@ -299,7 +315,7 @@ public static List filterIncrementalFileStatus(Job job, HoodieTableM for (HoodieBaseFile filteredFile : filteredFiles) { LOG.debug("Processing incremental hoodie file - " + filteredFile.getPath()); filteredFile = refreshFileStatus(job.getConfiguration(), filteredFile); - returns.add(filteredFile.getFileStatus()); + returns.add(getFileStatus(filteredFile)); } LOG.info("Total paths to process after hoodie incremental filter " + filteredFiles.size()); return returns; @@ -350,7 +366,7 @@ public static Map> groupFileStatusForSna * @return */ public static List filterFileStatusForSnapshotMode( - JobConf job, HoodieTableMetaClient metadata, List fileStatuses) { + JobConf job, HoodieTableMetaClient metadata, List fileStatuses) throws IOException { FileStatus[] statuses = fileStatuses.toArray(new FileStatus[0]); if (LOG.isDebugEnabled()) { LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata); @@ -367,7 +383,7 @@ public static List filterFileStatusForSnapshotMode( LOG.debug("Processing latest hoodie file - " + filteredFile.getPath()); } filteredFile = refreshFileStatus(job, filteredFile); - returns.add(filteredFile.getFileStatus()); + returns.add(getFileStatus(filteredFile)); } return returns; } @@ -386,7 +402,7 @@ private static HoodieBaseFile refreshFileStatus(Configuration conf, HoodieBaseFi if (dataFile.getFileSize() == 0) { FileSystem fs = dataPath.getFileSystem(conf); LOG.info("Refreshing file status " + dataFile.getPath()); - return new HoodieBaseFile(fs.getFileStatus(dataPath)); + return new HoodieBaseFile(fs.getFileStatus(dataPath), dataFile.getBootstrapBaseFile().orElse(null)); } return dataFile; } catch (IOException e) { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 7ae4ea0e5577c..cc46d96c2f3c4 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -29,18 +29,22 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.BootstrapBaseFileSplit; import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit; +import org.apache.hudi.hadoop.realtime.RealtimeBootstrapBaseFileSplit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.SplitLocationInfo; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -59,7 +63,7 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, Stream rtSplits = new ArrayList<>(); + List rtSplits = new ArrayList<>(); partitionsToParquetSplits.keySet().forEach(partitionPath -> { // for each partition path obtain the data & log file groupings, then map back to inputsplits HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath); @@ -78,19 +82,29 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, Stream> groupedInputSplits = partitionsToParquetSplits.get(partitionPath).stream() .collect(Collectors.groupingBy(split -> FSUtils.getFileId(split.getPath().getName()))); + // Get the maxCommit from the last delta or compaction or commit - when bootstrapped from COW table + String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, + HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) + .filterCompletedInstants().lastInstant().get().getTimestamp(); latestFileSlices.forEach(fileSlice -> { List dataFileSplits = groupedInputSplits.get(fileSlice.getFileId()); dataFileSplits.forEach(split -> { try { List logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()) .map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()); - // Get the maxCommit from the last delta or compaction or commit - when - // bootstrapped from COW table - String maxCommitTime = metaClient - .getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, - HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) - .filterCompletedInstants().lastInstant().get().getTimestamp(); - rtSplits.add(new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths, maxCommitTime)); + if (split instanceof BootstrapBaseFileSplit) { + BootstrapBaseFileSplit eSplit = (BootstrapBaseFileSplit) split; + String[] hosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo()) + .filter(x -> !x.isInMemory()).toArray(String[]::new) : new String[0]; + String[] inMemoryHosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo()) + .filter(SplitLocationInfo::isInMemory).toArray(String[]::new) : new String[0]; + FileSplit baseSplit = new FileSplit(eSplit.getPath(), eSplit.getStart(), eSplit.getLength(), + hosts, inMemoryHosts); + rtSplits.add(new RealtimeBootstrapBaseFileSplit(baseSplit,metaClient.getBasePath(), + logFilePaths, maxCommitTime, eSplit.getBootstrapFileSplit())); + } else { + rtSplits.add(new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths, maxCommitTime)); + } } catch (IOException e) { throw new HoodieIOException("Error creating hoodie real time split ", e); } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java index a489bacc30421..f28e6bfc031b1 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java @@ -23,6 +23,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.utilities.deltastreamer.DeltaSync; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.spark.api.java.JavaRDD; @@ -39,14 +40,9 @@ public HoodieDeltaStreamerWrapper(Config cfg, JavaSparkContext jssc) throws Exce super(cfg, jssc); } - public HoodieDeltaStreamerWrapper(Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf conf) throws Exception { - super(cfg, jssc, fs, conf); - } - - public JavaRDD upsert(Operation operation) throws - Exception { + public JavaRDD upsert(Operation operation) throws Exception { cfg.operation = operation; - return deltaSyncService.getDeltaSync().syncOnce().getRight(); + return deltaSyncService.get().getDeltaSync().syncOnce().getRight(); } public JavaRDD insert() throws Exception { @@ -71,7 +67,8 @@ public JavaRDD compact() throws Exception { } public Pair>> fetchSource() throws Exception { - return deltaSyncService.getDeltaSync().readFromSource(deltaSyncService.getDeltaSync().getCommitTimelineOpt()); + DeltaSync service = deltaSyncService.get().getDeltaSync(); + return service.readFromSource(service.getCommitTimelineOpt()); } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java index be036c16d49b3..25a694c5f1786 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java @@ -41,7 +41,7 @@ import org.apache.hudi.integ.testsuite.generator.DeltaGenerator; import org.apache.hudi.integ.testsuite.reader.DeltaInputType; import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode; -import org.apache.hudi.keygen.KeyGenerator; +import org.apache.hudi.keygen.BuiltinKeyGenerator; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; import org.apache.hudi.utilities.schema.SchemaProvider; @@ -85,7 +85,7 @@ public class HoodieTestSuiteJob { */ private transient HiveConf hiveConf; - private KeyGenerator keyGenerator; + private BuiltinKeyGenerator keyGenerator; public HoodieTestSuiteJob(HoodieTestSuiteConfig cfg, JavaSparkContext jsc) throws IOException { this.cfg = cfg; @@ -96,7 +96,8 @@ public HoodieTestSuiteJob(HoodieTestSuiteConfig cfg, JavaSparkContext jsc) throw log.info("Creating workload generator with configs : {}", props.toString()); this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jsc); this.hiveConf = getDefaultHiveConf(jsc.hadoopConfiguration()); - this.keyGenerator = DataSourceUtils.createKeyGenerator(props); + this.keyGenerator = (BuiltinKeyGenerator) DataSourceUtils.createKeyGenerator(props); + if (!fs.exists(new Path(cfg.targetBasePath))) { HoodieTableMetaClient.initTableType(jsc.hadoopConfiguration(), cfg.targetBasePath, HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived"); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java index 24311e0fe03d5..930f3078d3009 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java @@ -40,6 +40,7 @@ import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats; import org.apache.hudi.integ.testsuite.writer.DeltaWriterAdapter; import org.apache.hudi.integ.testsuite.writer.DeltaWriterFactory; +import org.apache.hudi.keygen.BuiltinKeyGenerator; import org.apache.hudi.keygen.ComplexKeyGenerator; import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator; @@ -70,16 +71,13 @@ public class DeltaGenerator implements Serializable { private int batchId; public DeltaGenerator(DeltaConfig deltaOutputConfig, JavaSparkContext jsc, SparkSession sparkSession, - String schemaStr, - KeyGenerator keyGenerator) { + String schemaStr, BuiltinKeyGenerator keyGenerator) { this.deltaOutputConfig = deltaOutputConfig; this.jsc = jsc; this.sparkSession = sparkSession; this.schemaStr = schemaStr; - this.recordRowKeyFieldNames = keyGenerator instanceof ComplexKeyGenerator ? ((ComplexKeyGenerator) keyGenerator) - .getRecordKeyFields() : Arrays.asList(((SimpleKeyGenerator) keyGenerator).getRecordKeyField()); - this.partitionPathFieldNames = keyGenerator instanceof ComplexKeyGenerator ? ((ComplexKeyGenerator) keyGenerator) - .getPartitionPathFields() : Arrays.asList(((SimpleKeyGenerator) keyGenerator).getPartitionPathField()); + this.recordRowKeyFieldNames = keyGenerator.getRecordKeyFields(); + this.partitionPathFieldNames = keyGenerator.getPartitionPathFields(); } public JavaRDD writeRecords(JavaRDD records) { diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java index fc4c259c41b6a..7b70857a9adcc 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java @@ -162,7 +162,9 @@ private TestExecStartResultCallback executeCommandInDocker(String containerName, .awaitCompletion(); int exitCode = dockerClient.inspectExecCmd(createCmdResponse.getId()).exec().getExitCode(); LOG.info("Exit code for command : " + exitCode); - LOG.error("\n\n ###### Stdout #######\n" + callback.getStdout().toString()); + if (exitCode != 0) { + LOG.error("\n\n ###### Stdout #######\n" + callback.getStdout().toString()); + } LOG.error("\n\n ###### Stderr #######\n" + callback.getStderr().toString()); if (expectedToSucceed) { @@ -240,6 +242,9 @@ private void saveUpLogs() { String filePath = System.getProperty("java.io.tmpdir") + "/" + System.currentTimeMillis() + "-hive.log"; FileIOUtils.writeStringToFile(hiveLogStr, filePath); LOG.info("Hive log saved up at : " + filePath); + LOG.info("<=========== Full hive log ===============>\n" + + "\n" + hiveLogStr + + "\n <==========================================>"); } catch (Exception e) { LOG.error("Unable to save up logs..", e); } @@ -268,7 +273,7 @@ void assertStdOutContains(Pair stdOutErr, String expectedOutput, saveUpLogs(); } - assertEquals(times, count, "Did not find output the expected number of times"); + assertEquals(times, count, "Did not find output the expected number of times."); } public class TestExecStartResultCallback extends ExecStartResultCallback { 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 dbbe0f191c712..d2a0841729a97 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 @@ -22,6 +22,7 @@ import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.keygen.SimpleKeyGenerator; import org.junit.jupiter.api.Test; import java.util.Collections; @@ -53,11 +54,18 @@ public class ITTestHoodieDemo extends ITTestBase { private static final String COW_TABLE_NAME = "stock_ticks_cow"; private static final String MOR_TABLE_NAME = "stock_ticks_mor"; + private static final String BOOTSTRAPPED_SRC_PATH = "/user/hive/warehouse/stock_ticks_cow_bs_src"; + private static final String COW_BOOTSTRAPPED_BASE_PATH = "/user/hive/warehouse/stock_ticks_cow_bs"; + private static final String MOR_BOOTSTRAPPED_BASE_PATH = "/user/hive/warehouse/stock_ticks_mor_bs"; + private static final String COW_BOOTSTRAPPED_TABLE_NAME = "stock_ticks_cow_bs"; + private static final String MOR_BOOTSTRAPPED_TABLE_NAME = "stock_ticks_mor_bs"; + private static final String DEMO_CONTAINER_SCRIPT = HOODIE_WS_ROOT + "/docker/demo/setup_demo_container.sh"; private static final String MIN_COMMIT_TIME_COW_SCRIPT = HOODIE_WS_ROOT + "/docker/demo/get_min_commit_time_cow.sh"; private static final String MIN_COMMIT_TIME_MOR_SCRIPT = HOODIE_WS_ROOT + "/docker/demo/get_min_commit_time_mor.sh"; private static final String HUDI_CLI_TOOL = HOODIE_WS_ROOT + "/hudi-cli/hudi-cli.sh"; private static final String COMPACTION_COMMANDS = HOODIE_WS_ROOT + "/docker/demo/compaction.commands"; + private static final String SPARKSQL_BS_PREP_COMMANDS = HOODIE_WS_ROOT + "/docker/demo/sparksql-bootstrap-prep-source.commands"; private static final String SPARKSQL_BATCH1_COMMANDS = HOODIE_WS_ROOT + "/docker/demo/sparksql-batch1.commands"; private static final String SPARKSQL_BATCH2_COMMANDS = HOODIE_WS_ROOT + "/docker/demo/sparksql-batch2.commands"; private static final String SPARKSQL_INCREMENTAL_COMMANDS = HOODIE_WS_ROOT + "/docker/demo/sparksql-incremental.commands"; @@ -100,6 +108,7 @@ public void testParquetDemo() throws Exception { // compaction scheduleAndRunCompaction(); + testHiveAfterSecondBatchAfterCompaction(); testPrestoAfterSecondBatchAfterCompaction(); testIncrementalHiveQueryAfterCompaction(); @@ -155,40 +164,80 @@ private void ingestFirstBatchAndHiveSync() throws Exception { + " --disable-compaction " + String.format(HIVE_SYNC_CMD_FMT, "dt", MOR_TABLE_NAME))); executeCommandStringsInDocker(ADHOC_1_CONTAINER, cmds); + executeSparkSQLCommand(SPARKSQL_BS_PREP_COMMANDS, true); + List bootstrapCmds = CollectionUtils.createImmutableList( + "spark-submit --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer " + HUDI_UTILITIES_BUNDLE + + " --table-type COPY_ON_WRITE " + + " --run-bootstrap " + + " --source-class org.apache.hudi.utilities.sources.JsonDFSSource --source-ordering-field ts " + + " --target-base-path " + COW_BOOTSTRAPPED_BASE_PATH + " --target-table " + COW_BOOTSTRAPPED_TABLE_NAME + + " --props /var/demo/config/dfs-source.properties" + + " --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider " + + " --initial-checkpoint-provider" + + " org.apache.hudi.utilities.checkpointing.InitialCheckpointFromAnotherHoodieTimelineProvider" + + " --hoodie-conf hoodie.bootstrap.base.path=" + BOOTSTRAPPED_SRC_PATH + + " --hoodie-conf hoodie.bootstrap.recordkey.columns=key" + + " --hoodie-conf hoodie.deltastreamer.checkpoint.provider.path=" + COW_BASE_PATH + + " --hoodie-conf hoodie.bootstrap.parallelism=2 " + + " --hoodie-conf hoodie.bootstrap.keygen.class=" + SimpleKeyGenerator.class.getName() + + String.format(HIVE_SYNC_CMD_FMT, "dt", COW_BOOTSTRAPPED_TABLE_NAME), + "spark-submit --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer " + HUDI_UTILITIES_BUNDLE + + " --table-type MERGE_ON_READ " + + " --run-bootstrap " + + " --source-class org.apache.hudi.utilities.sources.JsonDFSSource --source-ordering-field ts " + + " --target-base-path " + MOR_BOOTSTRAPPED_BASE_PATH + " --target-table " + MOR_BOOTSTRAPPED_TABLE_NAME + + " --props /var/demo/config/dfs-source.properties" + + " --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider " + + " --initial-checkpoint-provider" + + " org.apache.hudi.utilities.checkpointing.InitialCheckpointFromAnotherHoodieTimelineProvider" + + " --hoodie-conf hoodie.bootstrap.base.path=" + BOOTSTRAPPED_SRC_PATH + + " --hoodie-conf hoodie.bootstrap.recordkey.columns=key" + + " --hoodie-conf hoodie.deltastreamer.checkpoint.provider.path=" + COW_BASE_PATH + + " --hoodie-conf hoodie.bootstrap.parallelism=2 " + + " --hoodie-conf hoodie.bootstrap.keygen.class=" + SimpleKeyGenerator.class.getName() + + String.format(HIVE_SYNC_CMD_FMT, "dt", MOR_BOOTSTRAPPED_TABLE_NAME)); + executeCommandStringsInDocker(ADHOC_1_CONTAINER, bootstrapCmds); } private void testHiveAfterFirstBatch() throws Exception { Pair stdOutErrPair = executeHiveCommandFile(HIVE_TBLCHECK_COMMANDS); assertStdOutContains(stdOutErrPair, "| stock_ticks_cow |"); + assertStdOutContains(stdOutErrPair, "| stock_ticks_cow_bs |"); assertStdOutContains(stdOutErrPair, "| stock_ticks_mor_ro |"); assertStdOutContains(stdOutErrPair, "| stock_ticks_mor_rt |"); - + assertStdOutContains(stdOutErrPair, "| stock_ticks_mor_bs_ro |"); + assertStdOutContains(stdOutErrPair, "| stock_ticks_mor_bs_rt |"); assertStdOutContains(stdOutErrPair, "| partition |\n+----------------+\n| dt=2018-08-31 |\n+----------------+\n", 3); stdOutErrPair = executeHiveCommandFile(HIVE_BATCH1_COMMANDS); assertStdOutContains(stdOutErrPair, "| symbol | _c1 |\n+---------+----------------------+\n" - + "| GOOG | 2018-08-31 10:29:00 |\n", 3); + + "| GOOG | 2018-08-31 10:29:00 |\n", 6); assertStdOutContains(stdOutErrPair, "| symbol | ts | volume | open | close |\n" + "+---------+----------------------+---------+------------+-----------+\n" + "| GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 |\n" + "| GOOG | 2018-08-31 10:29:00 | 3391 | 1230.1899 | 1230.085 |\n", - 3); + 6); } private void testSparkSQLAfterFirstBatch() throws Exception { Pair stdOutErrPair = executeSparkSQLCommand(SPARKSQL_BATCH1_COMMANDS, true); assertStdOutContains(stdOutErrPair, "|default |stock_ticks_cow |false |\n" + + "|default |stock_ticks_cow_bs |false |\n" + + "|default |stock_ticks_mor_bs_ro |false |\n" + + "|default |stock_ticks_mor_bs_rt |false |" + "|default |stock_ticks_mor_ro |false |\n" - + "|default |stock_ticks_mor_rt |false |"); + + "|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); - assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 10:29:00|3391 |1230.1899|1230.085|", 3); + "+------+-------------------+\n|GOOG |2018-08-31 10:29:00|\n+------+-------------------+", 6); + assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 09:59:00|6330 |1230.5 |1230.02 |", 6); + assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 10:29:00|3391 |1230.1899|1230.085|", 6); } private void ingestSecondBatchAndHiveSync() throws Exception { + // Note : Unlike normal tables, bootstrapped tables do not have checkpoint. So, they + // begin with null checkpoint and read all states. List cmds = CollectionUtils.createImmutableList( ("hdfs dfs -copyFromLocal -f " + INPUT_BATCH_PATH2 + " " + HDFS_BATCH_PATH2), ("spark-submit" @@ -208,14 +257,28 @@ private void ingestSecondBatchAndHiveSync() throws Exception { + " --target-base-path " + MOR_BASE_PATH + " --target-table " + MOR_TABLE_NAME + " --props /var/demo/config/dfs-source.properties" + " --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider " - + " --disable-compaction " + String.format(HIVE_SYNC_CMD_FMT, "dt", MOR_TABLE_NAME))); + + " --disable-compaction " + String.format(HIVE_SYNC_CMD_FMT, "dt", MOR_TABLE_NAME)), + ("spark-submit --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer " + HUDI_UTILITIES_BUNDLE + + " --table-type COPY_ON_WRITE " + + " --source-class org.apache.hudi.utilities.sources.JsonDFSSource --source-ordering-field ts " + + " --target-base-path " + COW_BOOTSTRAPPED_BASE_PATH + " --target-table " + COW_BOOTSTRAPPED_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_BOOTSTRAPPED_TABLE_NAME)), + ("spark-submit --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer " + HUDI_UTILITIES_BUNDLE + + " --table-type MERGE_ON_READ " + + " --source-class org.apache.hudi.utilities.sources.JsonDFSSource --source-ordering-field ts " + + " --target-base-path " + MOR_BOOTSTRAPPED_BASE_PATH + " --target-table " + MOR_BOOTSTRAPPED_TABLE_NAME + + " --props /var/demo/config/dfs-source.properties" + + " --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider " + + " --disable-compaction " + String.format(HIVE_SYNC_CMD_FMT, "dt", MOR_BOOTSTRAPPED_TABLE_NAME))); executeCommandStringsInDocker(ADHOC_1_CONTAINER, cmds); } private void testPrestoAfterFirstBatch() throws Exception { Pair stdOutErrPair = executePrestoCommandFile(HDFS_PRESTO_INPUT_TABLE_CHECK_PATH); - assertStdOutContains(stdOutErrPair, "stock_ticks_cow"); - assertStdOutContains(stdOutErrPair, "stock_ticks_mor",2); + assertStdOutContains(stdOutErrPair, "stock_ticks_cow", 2); + assertStdOutContains(stdOutErrPair, "stock_ticks_mor",4); stdOutErrPair = executePrestoCommandFile(HDFS_PRESTO_INPUT_BATCH1_PATH); assertStdOutContains(stdOutErrPair, @@ -229,20 +292,20 @@ private void testPrestoAfterFirstBatch() throws Exception { private void testHiveAfterSecondBatch() throws Exception { Pair stdOutErrPair = executeHiveCommandFile(HIVE_BATCH1_COMMANDS); assertStdOutContains(stdOutErrPair, "| symbol | _c1 |\n+---------+----------------------+\n" - + "| GOOG | 2018-08-31 10:29:00 |\n"); + + "| GOOG | 2018-08-31 10:29:00 |\n", 2); assertStdOutContains(stdOutErrPair, "| symbol | _c1 |\n+---------+----------------------+\n" - + "| GOOG | 2018-08-31 10:59:00 |\n", 2); + + "| GOOG | 2018-08-31 10:59:00 |\n", 4); assertStdOutContains(stdOutErrPair, "| symbol | ts | volume | open | close |\n" + "+---------+----------------------+---------+------------+-----------+\n" + "| GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 |\n" - + "| GOOG | 2018-08-31 10:29:00 | 3391 | 1230.1899 | 1230.085 |\n"); + + "| GOOG | 2018-08-31 10:29:00 | 3391 | 1230.1899 | 1230.085 |\n", 2); assertStdOutContains(stdOutErrPair, "| symbol | ts | volume | open | close |\n" + "+---------+----------------------+---------+------------+-----------+\n" + "| GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 |\n" + "| GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 |\n", - 2); + 4); } private void testPrestoAfterSecondBatch() throws Exception { @@ -262,13 +325,13 @@ private void testPrestoAfterSecondBatch() throws Exception { private void testHiveAfterSecondBatchAfterCompaction() throws Exception { Pair stdOutErrPair = executeHiveCommandFile(HIVE_BATCH2_COMMANDS); assertStdOutContains(stdOutErrPair, "| symbol | _c1 |\n+---------+----------------------+\n" - + "| GOOG | 2018-08-31 10:59:00 |", 2); + + "| GOOG | 2018-08-31 10:59:00 |", 4); assertStdOutContains(stdOutErrPair, "| symbol | ts | volume | open | close |\n" + "+---------+----------------------+---------+------------+-----------+\n" + "| GOOG | 2018-08-31 09:59:00 | 6330 | 1230.5 | 1230.02 |\n" + "| GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 |", - 2); + 4); } private void testPrestoAfterSecondBatchAfterCompaction() throws Exception { @@ -284,13 +347,13 @@ private void testPrestoAfterSecondBatchAfterCompaction() throws Exception { private void testSparkSQLAfterSecondBatch() throws Exception { Pair stdOutErrPair = executeSparkSQLCommand(SPARKSQL_BATCH2_COMMANDS, true); assertStdOutContains(stdOutErrPair, - "+------+-------------------+\n|GOOG |2018-08-31 10:59:00|\n+------+-------------------+", 2); + "+------+-------------------+\n|GOOG |2018-08-31 10:59:00|\n+------+-------------------+", 4); - assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 09:59:00|6330 |1230.5 |1230.02 |", 3); - assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 10:59:00|9021 |1227.1993|1227.215|", 2); + assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 09:59:00|6330 |1230.5 |1230.02 |", 6); + assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 10:59:00|9021 |1227.1993|1227.215|", 4); assertStdOutContains(stdOutErrPair, - "+------+-------------------+\n|GOOG |2018-08-31 10:29:00|\n+------+-------------------+"); - assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 10:29:00|3391 |1230.1899|1230.085|"); + "+------+-------------------+\n|GOOG |2018-08-31 10:29:00|\n+------+-------------------+", 2); + assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 10:29:00|3391 |1230.1899|1230.085|", 2); } private void testIncrementalHiveQuery(String minCommitTimeScript, String incrementalCommandsFile, @@ -306,36 +369,40 @@ private void testIncrementalHiveQueryBeforeCompaction() throws Exception { String expectedOutput = "| GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 |"; // verify that 10:59 is present in COW table because there is no compaction process for COW - testIncrementalHiveQuery(MIN_COMMIT_TIME_COW_SCRIPT, HIVE_INCREMENTAL_COW_COMMANDS, expectedOutput, 1); + testIncrementalHiveQuery(MIN_COMMIT_TIME_COW_SCRIPT, HIVE_INCREMENTAL_COW_COMMANDS, expectedOutput, 2); // verify that 10:59 is NOT present in RO table because of pending compaction testIncrementalHiveQuery(MIN_COMMIT_TIME_MOR_SCRIPT, HIVE_INCREMENTAL_MOR_RO_COMMANDS, expectedOutput, 0); // verify that 10:59 is present in RT table even with pending compaction - testIncrementalHiveQuery(MIN_COMMIT_TIME_MOR_SCRIPT, HIVE_INCREMENTAL_MOR_RT_COMMANDS, expectedOutput, 1); + testIncrementalHiveQuery(MIN_COMMIT_TIME_MOR_SCRIPT, HIVE_INCREMENTAL_MOR_RT_COMMANDS, expectedOutput, 2); } private void testIncrementalHiveQueryAfterCompaction() throws Exception { String expectedOutput = "| symbol | ts | volume | open | close |\n" - + "+---------+----------------------+---------+------------+-----------+\n" - + "| GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 |"; + + "+---------+----------------------+---------+------------+-----------+\n" + + "| GOOG | 2018-08-31 10:59:00 | 9021 | 1227.1993 | 1227.215 |"; // verify that 10:59 is present for all views because compaction is complete - testIncrementalHiveQuery(MIN_COMMIT_TIME_COW_SCRIPT, HIVE_INCREMENTAL_COW_COMMANDS, expectedOutput, 1); - testIncrementalHiveQuery(MIN_COMMIT_TIME_MOR_SCRIPT, HIVE_INCREMENTAL_MOR_RO_COMMANDS, expectedOutput, 1); - testIncrementalHiveQuery(MIN_COMMIT_TIME_MOR_SCRIPT, HIVE_INCREMENTAL_MOR_RT_COMMANDS, expectedOutput, 1); + testIncrementalHiveQuery(MIN_COMMIT_TIME_COW_SCRIPT, HIVE_INCREMENTAL_COW_COMMANDS, expectedOutput, 2); + testIncrementalHiveQuery(MIN_COMMIT_TIME_MOR_SCRIPT, HIVE_INCREMENTAL_MOR_RO_COMMANDS, expectedOutput, 2); + testIncrementalHiveQuery(MIN_COMMIT_TIME_MOR_SCRIPT, HIVE_INCREMENTAL_MOR_RT_COMMANDS, expectedOutput, 2); } private void testIncrementalSparkSQLQuery() throws Exception { 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_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); + assertStdOutContains(stdOutErrPair, "|GOOG |2018-08-31 10:59:00|9021 |1227.1993|1227.215|", 2); + assertStdOutContains(stdOutErrPair, "|default |stock_ticks_cow |false |\n" + + "|default |stock_ticks_cow_bs |false |\n" + + "|default |stock_ticks_derived_mor_bs_ro|false |\n" + + "|default |stock_ticks_derived_mor_bs_rt|false |\n" + + "|default |stock_ticks_derived_mor_ro |false |\n" + + "|default |stock_ticks_derived_mor_rt |false |\n" + + "|default |stock_ticks_mor_bs_ro |false |\n" + + "|default |stock_ticks_mor_bs_rt |false |" + + "|default |stock_ticks_mor_ro |false |\n" + + "|default |stock_ticks_mor_rt |false |"); + assertStdOutContains(stdOutErrPair, "|count(1)|\n+--------+\n|99 |", 4); } private void scheduleAndRunCompaction() throws Exception { 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 5740f4b3e2d13..16d6f8d04e077 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -69,14 +69,6 @@ public class DataSourceUtils { private static final Logger LOG = LogManager.getLogger(DataSourceUtils.class); - /** - * Obtain value of the provided field as string, denoted by dot notation. e.g: a.b.c - */ - public static String getNestedFieldValAsString(GenericRecord record, String fieldName, boolean returnNullIfNotFound) { - Object obj = getNestedFieldVal(record, fieldName, returnNullIfNotFound); - return (obj == null) ? null : obj.toString(); - } - /** * Obtain value of the provided field, denoted by dot notation. e.g: a.b.c */ @@ -108,8 +100,8 @@ public static Object getNestedFieldVal(GenericRecord record, String fieldName, b return null; } else { throw new HoodieException( - fieldName + "(Part -" + parts[i] + ") field not found in record. Acceptable fields were :" - + valueNode.getSchema().getFields().stream().map(Field::name).collect(Collectors.toList())); + fieldName + "(Part -" + parts[i] + ") field not found in record. Acceptable fields were :" + + valueNode.getSchema().getFields().stream().map(Field::name).collect(Collectors.toList())); } } @@ -202,12 +194,12 @@ public static HoodieDateTimeParser createDateTimeParser(TypedProperties props, S * @see HoodieWriteConfig#getUserDefinedBulkInsertPartitionerClass() */ private static Option createUserDefinedBulkInsertPartitioner(HoodieWriteConfig config) - throws HoodieException { + throws HoodieException { String bulkInsertPartitionerClass = config.getUserDefinedBulkInsertPartitionerClass(); try { return StringUtils.isNullOrEmpty(bulkInsertPartitionerClass) - ? Option.empty() : - Option.of((BulkInsertPartitioner) ReflectionUtils.loadClass(bulkInsertPartitionerClass)); + ? Option.empty() : + Option.of((BulkInsertPartitioner) ReflectionUtils.loadClass(bulkInsertPartitionerClass)); } catch (Throwable e) { throw new HoodieException("Could not create UserDefinedBulkInsertPartitioner class " + bulkInsertPartitionerClass, e); } @@ -343,7 +335,7 @@ public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String b props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(), SlashEncodedDayPartitionValueExtractor.class.getName()); hiveSyncConfig.useJdbc = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_USE_JDBC_OPT_KEY(), - DataSourceWriteOptions.DEFAULT_HIVE_USE_JDBC_OPT_VAL())); + DataSourceWriteOptions.DEFAULT_HIVE_USE_JDBC_OPT_VAL())); return hiveSyncConfig; } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java b/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java new file mode 100644 index 0000000000000..0d5756f469456 --- /dev/null +++ b/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.bootstrap; + +import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; +import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.keygen.KeyGenerator; + +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; + +import java.io.IOException; +import java.util.List; + +/** + * Spark Data frame based bootstrap input provider. + */ +public class SparkParquetBootstrapDataProvider extends FullRecordBootstrapDataProvider { + + private final transient SparkSession sparkSession; + + public SparkParquetBootstrapDataProvider(TypedProperties props, + JavaSparkContext jsc) { + super(props, jsc); + this.sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate(); + } + + @Override + public JavaRDD generateInputRecordRDD(String tableName, String sourceBasePath, + List>> partitionPathsWithFiles) { + String[] filePaths = partitionPathsWithFiles.stream().map(Pair::getValue) + .flatMap(f -> f.stream().map(fs -> FileStatusUtils.toPath(fs.getPath()).toUri().getPath())) + .toArray(String[]::new); + Dataset inputDataset = sparkSession.read().parquet(filePaths); + try { + KeyGenerator keyGenerator = DataSourceUtils.createKeyGenerator(props); + String structName = tableName + "_record"; + String namespace = "hoodie." + tableName; + RDD genericRecords = AvroConversionUtils.createRdd(inputDataset, structName, namespace); + return genericRecords.toJavaRDD().map(gr -> { + String orderingVal = HoodieAvroUtils.getNestedFieldValAsString( + gr, props.getString("hoodie.datasource.write.precombine.field"), false); + try { + return DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr), + props.getString("hoodie.datasource.write.payload.class"), "_hoodie_is_deleted"); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + }); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } +} \ No newline at end of file diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java index b6431fe9a6d2f..e8996424d3cc7 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java @@ -18,11 +18,8 @@ package org.apache.hudi.keygen; -import org.apache.hudi.DataSourceUtils; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.exception.HoodieKeyException; import org.apache.avro.generic.GenericRecord; @@ -33,20 +30,14 @@ /** * Complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs. */ -public class ComplexKeyGenerator extends KeyGenerator { +public class ComplexKeyGenerator extends BuiltinKeyGenerator { - private static final String DEFAULT_PARTITION_PATH = "default"; - private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/"; public static final String DEFAULT_RECORD_KEY_SEPARATOR = ":"; - protected static final String NULL_RECORDKEY_PLACEHOLDER = "__null__"; - protected static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__"; - protected final List recordKeyFields; - protected final List partitionPathFields; - protected final boolean hiveStylePartitioning; + protected final boolean encodePartitionPath; public ComplexKeyGenerator(TypedProperties props) { super(props); @@ -55,59 +46,26 @@ public ComplexKeyGenerator(TypedProperties props) { Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList()); this.hiveStylePartitioning = props.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(), Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL())); + this.encodePartitionPath = props.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(), + Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL())); } @Override - public HoodieKey getKey(GenericRecord record) { - String recordKey = getRecordKey(record); - StringBuilder partitionPath = new StringBuilder(); - for (String partitionPathField : partitionPathFields) { - partitionPath.append(getPartitionPath(record, partitionPathField)); - partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR); - } - partitionPath.deleteCharAt(partitionPath.length() - 1); - - return new HoodieKey(recordKey, partitionPath.toString()); + public String getRecordKey(GenericRecord record) { + return KeyGenUtils.getRecordKey(record, recordKeyFields); } - String getPartitionPath(GenericRecord record, String partitionPathField) { - StringBuilder partitionPath = new StringBuilder(); - String fieldVal = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField, true); - if (fieldVal == null || fieldVal.isEmpty()) { - partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + DEFAULT_PARTITION_PATH - : DEFAULT_PARTITION_PATH); - } else { - partitionPath.append(hiveStylePartitioning ? partitionPathField + "=" + fieldVal : fieldVal); - } - return partitionPath.toString(); - } - - String getRecordKey(GenericRecord record) { - boolean keyIsNullEmpty = true; - StringBuilder recordKey = new StringBuilder(); - for (String recordKeyField : recordKeyFields) { - String recordKeyValue = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true); - if (recordKeyValue == null) { - recordKey.append(recordKeyField + ":" + NULL_RECORDKEY_PLACEHOLDER + ","); - } else if (recordKeyValue.isEmpty()) { - recordKey.append(recordKeyField + ":" + EMPTY_RECORDKEY_PLACEHOLDER + ","); - } else { - recordKey.append(recordKeyField + ":" + recordKeyValue + ","); - keyIsNullEmpty = false; - } - } - recordKey.deleteCharAt(recordKey.length() - 1); - if (keyIsNullEmpty) { - throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: " - + recordKeyFields.toString() + " cannot be entirely null or empty."); - } - return recordKey.toString(); + @Override + public String getPartitionPath(GenericRecord record) { + return KeyGenUtils.getRecordPartitionPath(record, partitionPathFields, hiveStylePartitioning, encodePartitionPath); } + @Override public List getRecordKeyFields() { return recordKeyFields; } + @Override public List getPartitionPathFields() { return partitionPathFields; } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java index 6240f837ebc1d..e5cc61c0a3cce 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java @@ -19,7 +19,6 @@ package org.apache.hudi.keygen; import org.apache.hudi.DataSourceWriteOptions; -import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.config.TypedProperties; import org.apache.avro.generic.GenericRecord; @@ -47,7 +46,7 @@ * * RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator. */ -public class CustomKeyGenerator extends KeyGenerator { +public class CustomKeyGenerator extends BuiltinKeyGenerator { protected final List recordKeyFields; protected final List partitionPathFields; @@ -71,15 +70,7 @@ public CustomKeyGenerator(TypedProperties props) { } @Override - public HoodieKey getKey(GenericRecord record) { - //call function to get the record key - String recordKey = getRecordKey(record); - //call function to get the partition key based on the type for that partition path field - String partitionPath = getPartitionPath(record); - return new HoodieKey(recordKey, partitionPath); - } - - private String getPartitionPath(GenericRecord record) { + public String getPartitionPath(GenericRecord record) { if (partitionPathFields == null) { throw new HoodieKeyException("Unable to find field names for partition path in cfg"); } @@ -101,11 +92,11 @@ private String getPartitionPath(GenericRecord record) { PartitionKeyType keyType = PartitionKeyType.valueOf(fieldWithType[1].toUpperCase()); switch (keyType) { case SIMPLE: - partitionPath.append(new SimpleKeyGenerator(properties).getPartitionPath(record, partitionPathField)); + partitionPath.append(new SimpleKeyGenerator(properties, partitionPathField).getPartitionPath(record)); break; case TIMESTAMP: try { - partitionPath.append(new TimestampBasedKeyGenerator(properties).getPartitionPath(record, partitionPathField)); + partitionPath.append(new TimestampBasedKeyGenerator(properties, partitionPathField).getPartitionPath(record)); } catch (IOException ioe) { throw new HoodieDeltaStreamerException("Unable to initialise TimestampBasedKeyGenerator class"); } @@ -121,11 +112,22 @@ private String getPartitionPath(GenericRecord record) { return partitionPath.toString(); } - private String getRecordKey(GenericRecord record) { + @Override + public String getRecordKey(GenericRecord record) { if (recordKeyFields == null || recordKeyFields.isEmpty()) { throw new HoodieKeyException("Unable to find field names for record key in cfg"); } return recordKeyFields.size() == 1 ? new SimpleKeyGenerator(properties).getRecordKey(record) : new ComplexKeyGenerator(properties).getRecordKey(record); } + + @Override + public List getRecordKeyFields() { + return recordKeyFields; + } + + @Override + public List getPartitionPathFields() { + return partitionPathFields; + } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java index 37b05291d21fc..5851a9dc7c28a 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java @@ -18,14 +18,12 @@ package org.apache.hudi.keygen; -import org.apache.hudi.DataSourceUtils; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.exception.HoodieKeyException; import org.apache.avro.generic.GenericRecord; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; @@ -34,11 +32,9 @@ * Key generator for deletes using global indices. Global index deletes do not require partition value * so this key generator avoids using partition value for generating HoodieKey. */ -public class GlobalDeleteKeyGenerator extends KeyGenerator { +public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator { private static final String EMPTY_PARTITION = ""; - private static final String NULL_RECORDKEY_PLACEHOLDER = "__null__"; - private static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__"; protected final List recordKeyFields; @@ -48,30 +44,22 @@ public GlobalDeleteKeyGenerator(TypedProperties config) { } @Override - public HoodieKey getKey(GenericRecord record) { - return new HoodieKey(getRecordKey(record), EMPTY_PARTITION); + public String getRecordKey(GenericRecord record) { + return KeyGenUtils.getRecordKey(record, recordKeyFields); } - String getRecordKey(GenericRecord record) { - boolean keyIsNullEmpty = true; - StringBuilder recordKey = new StringBuilder(); - for (String recordKeyField : recordKeyFields) { - String recordKeyValue = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true); - if (recordKeyValue == null) { - recordKey.append(recordKeyField + ":" + NULL_RECORDKEY_PLACEHOLDER + ","); - } else if (recordKeyValue.isEmpty()) { - recordKey.append(recordKeyField + ":" + EMPTY_RECORDKEY_PLACEHOLDER + ","); - } else { - recordKey.append(recordKeyField + ":" + recordKeyValue + ","); - keyIsNullEmpty = false; - } - } - recordKey.deleteCharAt(recordKey.length() - 1); - if (keyIsNullEmpty) { - throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: " - + recordKeyFields.toString() + " cannot be entirely null or empty."); - } + @Override + public String getPartitionPath(GenericRecord record) { + return EMPTY_PARTITION; + } + + @Override + public List getRecordKeyFields() { + return recordKeyFields; + } - return recordKey.toString(); + @Override + public List getPartitionPathFields() { + return new ArrayList<>(); } -} +} \ No newline at end of file diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java index bb2642cc6d7af..de4f50bf93df7 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java @@ -19,17 +19,17 @@ package org.apache.hudi.keygen; import org.apache.hudi.DataSourceWriteOptions; -import org.apache.hudi.DataSourceUtils; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.exception.HoodieKeyException; import org.apache.avro.generic.GenericRecord; +import java.util.ArrayList; +import java.util.List; + /** * Simple Key generator for unpartitioned Hive Tables. */ -public class NonpartitionedKeyGenerator extends KeyGenerator { +public class NonpartitionedKeyGenerator extends SimpleKeyGenerator { private static final String EMPTY_PARTITION = ""; @@ -41,11 +41,12 @@ public NonpartitionedKeyGenerator(TypedProperties props) { } @Override - public HoodieKey getKey(GenericRecord record) { - String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true); - if (recordKey == null || recordKey.isEmpty()) { - throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty."); - } - return new HoodieKey(recordKey, EMPTY_PARTITION); + public String getPartitionPath(GenericRecord record) { + return EMPTY_PARTITION; + } + + @Override + public List getPartitionPathFields() { + return new ArrayList<>(); } } \ No newline at end of file diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java index a9048600e961a..ea460b5ff980d 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java @@ -18,20 +18,18 @@ package org.apache.hudi.keygen; -import org.apache.hudi.DataSourceUtils; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.exception.HoodieKeyException; import org.apache.avro.generic.GenericRecord; +import java.util.Arrays; +import java.util.List; + /** * Simple key generator, which takes names of fields to be used for recordKey and partitionPath as configs. */ -public class SimpleKeyGenerator extends KeyGenerator { - - private static final String DEFAULT_PARTITION_PATH = "default"; +public class SimpleKeyGenerator extends BuiltinKeyGenerator { protected final String recordKeyField; @@ -39,46 +37,39 @@ public class SimpleKeyGenerator extends KeyGenerator { protected final boolean hiveStylePartitioning; + protected final boolean encodePartitionPath; + public SimpleKeyGenerator(TypedProperties props) { + this(props, props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY())); + } + + public SimpleKeyGenerator(TypedProperties props, String partitionPathField) { super(props); this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()); - this.partitionPathField = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()); + this.partitionPathField = partitionPathField; this.hiveStylePartitioning = props.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(), Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL())); + this.encodePartitionPath = props.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(), + Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL())); } @Override - public HoodieKey getKey(GenericRecord record) { - String recordKey = getRecordKey(record); - String partitionPath = getPartitionPath(record, partitionPathField); - return new HoodieKey(recordKey, partitionPath); + public String getRecordKey(GenericRecord record) { + return KeyGenUtils.getRecordKey(record, recordKeyField); } - String getPartitionPath(GenericRecord record, String partitionPathField) { - String partitionPath = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField, true); - if (partitionPath == null || partitionPath.isEmpty()) { - partitionPath = DEFAULT_PARTITION_PATH; - } - if (hiveStylePartitioning) { - partitionPath = partitionPathField + "=" + partitionPath; - } - - return partitionPath; - } - - String getRecordKey(GenericRecord record) { - String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField, true); - if (recordKey == null || recordKey.isEmpty()) { - throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty."); - } - return recordKey; + @Override + public String getPartitionPath(GenericRecord record) { + return KeyGenUtils.getPartitionPath(record, partitionPathField, hiveStylePartitioning, encodePartitionPath); } - public String getRecordKeyField() { - return recordKeyField; + @Override + public List getRecordKeyFields() { + return Arrays.asList(recordKeyField); } - public String getPartitionPathField() { - return partitionPathField; + @Override + public List getPartitionPathFields() { + return Arrays.asList(partitionPathField); } } \ No newline at end of file diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java index 894df433a80bb..b5e6fe8ec0df9 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java +++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java @@ -19,9 +19,11 @@ package org.apache.hudi.keygen; import org.apache.hudi.DataSourceUtils; -import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.exception.HoodieDeltaStreamerException; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.avro.generic.GenericRecord; @@ -34,6 +36,9 @@ import java.io.IOException; import java.io.Serializable; +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; import java.util.concurrent.TimeUnit; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -58,6 +63,8 @@ public enum TimestampType implements Serializable { // https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html private final DateTimeZone outputDateTimeZone; + protected final boolean encodePartitionPath; + /** * Supported configs. */ @@ -82,7 +89,11 @@ public static class Config { } public TimestampBasedKeyGenerator(TypedProperties config) throws IOException { - super(config); + this(config, config.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY())); + } + + public TimestampBasedKeyGenerator(TypedProperties config, String partitionPathField) throws IOException { + super(config, partitionPathField); String dateTimeParserClass = config.getString(Config.DATE_TIME_PARSER_PROP, HoodieDateTimeParserImpl.class.getName()); this.parser = DataSourceUtils.createDateTimeParser(config, dateTimeParserClass); this.outputDateTimeZone = parser.getOutputDateTimeZone(); @@ -108,17 +119,13 @@ public TimestampBasedKeyGenerator(TypedProperties config) throws IOException { default: timeUnit = null; } + this.encodePartitionPath = config.getBoolean(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY(), + Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL())); } @Override - public HoodieKey getKey(GenericRecord record) { - String recordKey = getRecordKey(record); - String partitionPath = getPartitionPath(record, partitionPathField); - return new HoodieKey(recordKey, partitionPath); - } - - String getPartitionPath(GenericRecord record, String partitionPathField) { - Object partitionVal = DataSourceUtils.getNestedFieldVal(record, partitionPathField, true); + public String getPartitionPath(GenericRecord record) { + Object partitionVal = HoodieAvroUtils.getNestedFieldVal(record, partitionPathField, true); if (partitionVal == null) { partitionVal = 1L; } @@ -146,11 +153,18 @@ String getPartitionPath(GenericRecord record, String partitionPathField) { timeMs = inputFormatter.parseDateTime(partitionVal.toString()).getMillis(); } else { throw new HoodieNotSupportedException( - "Unexpected type for partition field: " + partitionVal.getClass().getName()); + "Unexpected type for partition field: " + partitionVal.getClass().getName()); } DateTime timestamp = new DateTime(timeMs, outputDateTimeZone); - return hiveStylePartitioning ? partitionPathField + "=" + timestamp.toString(partitionFormatter) - : timestamp.toString(partitionFormatter); + String partitionPath = timestamp.toString(partitionFormatter); + if (encodePartitionPath) { + try { + partitionPath = URLEncoder.encode(partitionPath, StandardCharsets.UTF_8.toString()); + } catch (UnsupportedEncodingException uoe) { + throw new HoodieException(uoe.getMessage(), uoe); + } + } + return hiveStylePartitioning ? partitionPathField + "=" + partitionPath : partitionPath; } catch (Exception e) { throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, e); } 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 bb26302b91479..e529a045d6443 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -214,7 +214,8 @@ object DataSourceWriteOptions { */ val HIVE_STYLE_PARTITIONING_OPT_KEY = "hoodie.datasource.write.hive_style_partitioning" val DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL = "false" - + val URL_ENCODE_PARTITIONING_OPT_KEY = "hoodie.datasource.write.partitionpath.urlencode" + val DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL = "false" /** * Key generator class, that implements will extract the key out of incoming record * 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 f8f388e248ba1..574c9076c11b5 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -24,10 +24,11 @@ import org.apache.avro.generic.GenericRecord import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hudi.DataSourceWriteOptions._ +import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.{HoodieWriteClient, WriteStatus} import org.apache.hudi.common.config.TypedProperties import org.apache.hudi.common.fs.FSUtils -import org.apache.hudi.common.model.HoodieRecordPayload +import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.config.HoodieWriteConfig @@ -107,7 +108,7 @@ private[hudi] object HoodieSparkSqlWriter { val keyGenerator = DataSourceUtils.createKeyGenerator(toProperties(parameters)) val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace) val hoodieAllIncomingRecords = genericRecords.map(gr => { - val orderingVal = DataSourceUtils.getNestedFieldVal(gr, parameters(PRECOMBINE_FIELD_OPT_KEY), false) + val orderingVal = HoodieAvroUtils.getNestedFieldVal(gr, parameters(PRECOMBINE_FIELD_OPT_KEY), false) .asInstanceOf[Comparable[_]] DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr), @@ -131,8 +132,9 @@ private[hudi] object HoodieSparkSqlWriter { // Create the table if not present if (!exists) { - HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, tableType, - tblName, "archived", parameters(PAYLOAD_CLASS_OPT_KEY)) + //FIXME(bootstrap): bootstrapIndexClass needs to be set when bootstrap index class is integrated. + HoodieTableMetaClient.initTableTypeWithBootstrap(sparkContext.hadoopConfiguration, path.get, HoodieTableType.valueOf(tableType), + tblName, "archived", parameters(PAYLOAD_CLASS_OPT_KEY), null, null, null) } // Create a HoodieWriteClient & issue the write. diff --git a/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java b/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java index a5368512085b2..2cd4c422236eb 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java +++ b/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java @@ -18,6 +18,7 @@ package org.apache.hudi; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.HoodieWriteClient; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -92,14 +93,14 @@ public void testAvroRecordsFieldConversion() { record.put("event_name", "Hudi Meetup"); record.put("event_organizer", "Hudi PMC"); - assertEquals(LocalDate.ofEpochDay(18000).toString(), DataSourceUtils.getNestedFieldValAsString(record, "event_date1", + assertEquals(LocalDate.ofEpochDay(18000).toString(), HoodieAvroUtils.getNestedFieldValAsString(record, "event_date1", true)); - assertEquals(LocalDate.ofEpochDay(18001).toString(), DataSourceUtils.getNestedFieldValAsString(record, "event_date2", + assertEquals(LocalDate.ofEpochDay(18001).toString(), HoodieAvroUtils.getNestedFieldValAsString(record, "event_date2", true)); - assertEquals(LocalDate.ofEpochDay(18002).toString(), DataSourceUtils.getNestedFieldValAsString(record, "event_date3", + assertEquals(LocalDate.ofEpochDay(18002).toString(), HoodieAvroUtils.getNestedFieldValAsString(record, "event_date3", true)); - assertEquals("Hudi Meetup", DataSourceUtils.getNestedFieldValAsString(record, "event_name", true)); - assertEquals("Hudi PMC", DataSourceUtils.getNestedFieldValAsString(record, "event_organizer", true)); + assertEquals("Hudi Meetup", HoodieAvroUtils.getNestedFieldValAsString(record, "event_name", true)); + assertEquals("Hudi PMC", HoodieAvroUtils.getNestedFieldValAsString(record, "event_organizer", true)); } @Test diff --git a/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java b/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java new file mode 100644 index 0000000000000..88f535b17f5af --- /dev/null +++ b/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java @@ -0,0 +1,589 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client; + +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; +import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector; +import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector; +import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector; +import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.bootstrap.index.BootstrapIndex; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.testutils.RawTripTestPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ParquetReaderIterator; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieBootstrapConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.HoodieParquetInputFormat; +import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.io.DoubleWritable; +import org.apache.hudi.index.HoodieIndex.IndexType; +import org.apache.hudi.keygen.NonpartitionedKeyGenerator; +import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.table.action.bootstrap.BootstrapUtils; +import org.apache.hudi.testutils.HoodieClientTestBase; +import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.avro.AvroReadSupport; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.api.java.UDF1; +import org.apache.spark.sql.types.DataTypes; + +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.Map; +import java.util.Random; +import java.util.Spliterators; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.StreamSupport; + +import static java.util.stream.Collectors.mapping; +import static java.util.stream.Collectors.toList; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.generateGenericRecord; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.apache.spark.sql.functions.callUDF; + +/** + * Tests Bootstrap Client functionality. + */ +public class TestBootstrap extends HoodieClientTestBase { + + public static final String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double," + + "struct,array>,boolean"; + + @TempDir + public java.nio.file.Path tmpFolder; + + protected String bootstrapBasePath = null; + + private HoodieParquetInputFormat roInputFormat; + private JobConf roJobConf; + + private HoodieParquetRealtimeInputFormat rtInputFormat; + private JobConf rtJobConf; + private SparkSession spark; + + @BeforeEach + public void setUp() throws Exception { + bootstrapBasePath = tmpFolder.toAbsolutePath().toString() + "/data"; + initPath(); + spark = SparkSession.builder() + .appName("Bootstrap test") + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .getOrCreate(); + jsc = new JavaSparkContext(spark.sparkContext()); + sqlContext = spark.sqlContext(); + hadoopConf = spark.sparkContext().hadoopConfiguration(); + initTestDataGenerator(); + initMetaClient(); + // initialize parquet input format + reloadInputFormats(); + } + + @AfterEach + public void tearDown() throws IOException { + cleanupClients(); + cleanupTestDataGenerator(); + } + + private void reloadInputFormats() { + // initialize parquet input format + roInputFormat = new HoodieParquetInputFormat(); + roJobConf = new JobConf(jsc.hadoopConfiguration()); + roInputFormat.setConf(roJobConf); + + rtInputFormat = new HoodieParquetRealtimeInputFormat(); + rtJobConf = new JobConf(jsc.hadoopConfiguration()); + rtInputFormat.setConf(rtJobConf); + } + + public Schema generateNewDataSetAndReturnSchema(double timestamp, int numRecords, List partitionPaths, + String srcPath) throws Exception { + boolean isPartitioned = partitionPaths != null && !partitionPaths.isEmpty(); + Dataset df = generateTestRawTripDataset(timestamp, numRecords, partitionPaths, jsc, sqlContext); + df.printSchema(); + if (isPartitioned) { + df.write().partitionBy("datestr").format("parquet").mode(SaveMode.Overwrite).save(srcPath); + } else { + df.write().format("parquet").mode(SaveMode.Overwrite).save(srcPath); + } + String filePath = FileStatusUtils.toPath(BootstrapUtils.getAllLeafFoldersWithFiles(metaClient.getFs(), srcPath, + (status) -> status.getName().endsWith(".parquet")).stream().findAny().map(p -> p.getValue().stream().findAny()) + .orElse(null).get().getPath()).toString(); + ParquetFileReader reader = ParquetFileReader.open(metaClient.getHadoopConf(), new Path(filePath)); + MessageType schema = reader.getFooter().getFileMetaData().getSchema(); + return new AvroSchemaConverter().convert(schema); + } + + @Test + public void testMetadataBootstrapUnpartitionedCOW() throws Exception { + testBootstrapCommon(false, false, EffectiveMode.METADATA_BOOTSTRAP_MODE); + } + + @Test + public void testMetadataBootstrapWithUpdatesCOW() throws Exception { + testBootstrapCommon(true, false, EffectiveMode.METADATA_BOOTSTRAP_MODE); + } + + private enum EffectiveMode { + FULL_BOOTSTRAP_MODE, + METADATA_BOOTSTRAP_MODE, + MIXED_BOOTSTRAP_MODE + } + + private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, EffectiveMode mode) throws Exception { + + if (deltaCommit) { + metaClient = HoodieTestUtils.init(basePath, HoodieTableType.MERGE_ON_READ, bootstrapBasePath); + } else { + metaClient = HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, bootstrapBasePath); + } + + int totalRecords = 100; + String keyGeneratorClass = partitioned ? SimpleKeyGenerator.class.getCanonicalName() + : NonpartitionedKeyGenerator.class.getCanonicalName(); + final String bootstrapModeSelectorClass; + final String bootstrapCommitInstantTs; + final boolean checkNumRawFiles; + final boolean isBootstrapIndexCreated; + final int numInstantsAfterBootstrap; + final List bootstrapInstants; + switch (mode) { + case FULL_BOOTSTRAP_MODE: + bootstrapModeSelectorClass = FullRecordBootstrapModeSelector.class.getCanonicalName(); + bootstrapCommitInstantTs = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS; + checkNumRawFiles = false; + isBootstrapIndexCreated = false; + numInstantsAfterBootstrap = 1; + bootstrapInstants = Arrays.asList(bootstrapCommitInstantTs); + break; + case METADATA_BOOTSTRAP_MODE: + bootstrapModeSelectorClass = MetadataOnlyBootstrapModeSelector.class.getCanonicalName(); + bootstrapCommitInstantTs = HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS; + checkNumRawFiles = true; + isBootstrapIndexCreated = true; + numInstantsAfterBootstrap = 1; + bootstrapInstants = Arrays.asList(bootstrapCommitInstantTs); + break; + default: + bootstrapModeSelectorClass = TestRandomBootstapModeSelector.class.getName(); + bootstrapCommitInstantTs = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS; + checkNumRawFiles = false; + isBootstrapIndexCreated = true; + numInstantsAfterBootstrap = 2; + bootstrapInstants = Arrays.asList(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, + HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); + break; + } + List partitions = Arrays.asList("2020/04/01", "2020/04/02", "2020/04/03"); + double timestamp = new Double(Instant.now().toEpochMilli()).longValue(); + Schema schema = generateNewDataSetAndReturnSchema(timestamp, totalRecords, partitions, bootstrapBasePath); + HoodieWriteConfig config = getConfigBuilder(schema.toString()) + .withAutoCommit(true) + .withSchema(schema.toString()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(1) + .build()) + .withBootstrapConfig(HoodieBootstrapConfig.newBuilder() + .withBootstrapBasePath(bootstrapBasePath) + .withBootstrapKeyGenClass(keyGeneratorClass) + .withFullBootstrapInputProvider(TestFullBootstrapDataProvider.class.getName()) + .withBootstrapParallelism(3) + .withBootstrapModeSelector(bootstrapModeSelectorClass).build()) + .build(); + HoodieWriteClient client = new HoodieWriteClient(jsc, config); + client.bootstrap(Option.empty()); + checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, + numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); + + // Rollback Bootstrap + FSUtils.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED, + deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, bootstrapCommitInstantTs)); + client.rollBackInflightBootstrap(); + metaClient.reloadActiveTimeline(); + assertEquals(0, metaClient.getCommitsTimeline().countInstants()); + assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient.getFs(), basePath, + (status) -> status.getName().endsWith(".parquet")).stream().flatMap(f -> f.getValue().stream()).count()); + + BootstrapIndex index = BootstrapIndex.getBootstrapIndex(metaClient); + assertFalse(index.useIndex()); + + // Run bootstrap again + client = new HoodieWriteClient(jsc, config); + client.bootstrap(Option.empty()); + + metaClient.reloadActiveTimeline(); + index = BootstrapIndex.getBootstrapIndex(metaClient); + if (isBootstrapIndexCreated) { + assertTrue(index.useIndex()); + } else { + assertFalse(index.useIndex()); + } + + checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, + numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); + + // Upsert case + double updateTimestamp = new Double(Instant.now().toEpochMilli()).longValue(); + String updateSPath = tmpFolder.toAbsolutePath().toString() + "/data2"; + generateNewDataSetAndReturnSchema(updateTimestamp, totalRecords, partitions, updateSPath); + JavaRDD updateBatch = + generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient.getFs(), updateSPath, + (status) -> status.getName().endsWith("parquet")), schema); + String newInstantTs = client.startCommit(); + client.upsert(updateBatch, newInstantTs); + checkBootstrapResults(totalRecords, schema, newInstantTs, false, numInstantsAfterBootstrap + 1, + updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit); + + if (deltaCommit) { + Option compactionInstant = client.scheduleCompaction(Option.empty()); + assertTrue(compactionInstant.isPresent()); + client.compact(compactionInstant.get()); + checkBootstrapResults(totalRecords, schema, compactionInstant.get(), checkNumRawFiles, + numInstantsAfterBootstrap + 2, 2, updateTimestamp, updateTimestamp, !deltaCommit, + Arrays.asList(compactionInstant.get())); + } + } + + @Test + public void testMetadataBootstrapWithUpdatesMOR() throws Exception { + testBootstrapCommon(true, true, EffectiveMode.METADATA_BOOTSTRAP_MODE); + } + + @Test + public void testFullBoostrapOnlyCOW() throws Exception { + testBootstrapCommon(true, false, EffectiveMode.FULL_BOOTSTRAP_MODE); + } + + @Test + public void testFullBootstrapWithUpdatesMOR() throws Exception { + testBootstrapCommon(true, true, EffectiveMode.FULL_BOOTSTRAP_MODE); + } + + @Test + public void testMetaAndFullBoostrapCOW() throws Exception { + testBootstrapCommon(true, false, EffectiveMode.MIXED_BOOTSTRAP_MODE); + } + + @Test + public void testMetadataAndFullBootstrapWithUpdatesMOR() throws Exception { + testBootstrapCommon(true, true, EffectiveMode.MIXED_BOOTSTRAP_MODE); + } + + private void checkBootstrapResults(int totalRecords, Schema schema, String maxInstant, boolean checkNumRawFiles, + int expNumInstants, double expTimestamp, double expROTimestamp, boolean isDeltaCommit) throws Exception { + checkBootstrapResults(totalRecords, schema, maxInstant, checkNumRawFiles, expNumInstants, expNumInstants, + expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant)); + } + + private void checkBootstrapResults(int totalRecords, Schema schema, String instant, boolean checkNumRawFiles, + int expNumInstants, int numVersions, double expTimestamp, double expROTimestamp, boolean isDeltaCommit, + List instantsWithValidRecords) throws Exception { + metaClient.reloadActiveTimeline(); + assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants()); + assertEquals(instant, metaClient.getActiveTimeline() + .getCommitsTimeline().filterCompletedInstants().lastInstant().get().getTimestamp()); + + Dataset bootstrapped = sqlContext.read().format("parquet").load(basePath); + Dataset original = sqlContext.read().format("parquet").load(bootstrapBasePath); + bootstrapped.registerTempTable("bootstrapped"); + original.registerTempTable("original"); + if (checkNumRawFiles) { + List files = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient.getFs(), bootstrapBasePath, + (status) -> status.getName().endsWith(".parquet")) + .stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList()); + assertEquals(files.size() * numVersions, + sqlContext.sql("select distinct _hoodie_file_name from bootstrapped").count()); + } + + if (!isDeltaCommit) { + String predicate = String.join(", ", + instantsWithValidRecords.stream().map(p -> "\"" + p + "\"").collect(Collectors.toList())); + assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN " + + "(" + predicate + ")").count()); + Dataset missingOriginal = sqlContext.sql("select a._row_key from original a where a._row_key not " + + "in (select _hoodie_record_key from bootstrapped)"); + assertEquals(0, missingOriginal.count()); + Dataset missingBootstrapped = sqlContext.sql("select a._hoodie_record_key from bootstrapped a " + + "where a._hoodie_record_key not in (select _row_key from original)"); + assertEquals(0, missingBootstrapped.count()); + //sqlContext.sql("select * from bootstrapped").show(10, false); + } + + // RO Input Format Read + reloadInputFormats(); + List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); + assertEquals(totalRecords, records.size()); + Set seenKeys = new HashSet<>(); + for (GenericRecord r : records) { + assertEquals(r.get("_row_key").toString(), r.get("_hoodie_record_key").toString(), "Record :" + r); + assertEquals(expROTimestamp, ((DoubleWritable)r.get("timestamp")).get(), 0.1, "Record :" + r); + assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); + seenKeys.add(r.get("_hoodie_record_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + + //RT Input Format Read + reloadInputFormats(); + seenKeys = new HashSet<>(); + records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); + assertEquals(totalRecords, records.size()); + for (GenericRecord r : records) { + assertEquals(r.get("_row_key").toString(), r.get("_hoodie_record_key").toString(), "Realtime Record :" + r); + assertEquals(expTimestamp, ((DoubleWritable)r.get("timestamp")).get(),0.1, "Realtime Record :" + r); + assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); + seenKeys.add(r.get("_hoodie_record_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + + // RO Input Format Read - Project only Hoodie Columns + reloadInputFormats(); + records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, + true, HoodieRecord.HOODIE_META_COLUMNS); + assertEquals(totalRecords, records.size()); + seenKeys = new HashSet<>(); + for (GenericRecord r : records) { + assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); + seenKeys.add(r.get("_hoodie_record_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + + //RT Input Format Read - Project only Hoodie Columns + reloadInputFormats(); + seenKeys = new HashSet<>(); + records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, + HoodieRecord.HOODIE_META_COLUMNS); + assertEquals(totalRecords, records.size()); + for (GenericRecord r : records) { + assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); + seenKeys.add(r.get("_hoodie_record_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + + // RO Input Format Read - Project only non-hoodie column + reloadInputFormats(); + records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, + Arrays.asList("_row_key")); + assertEquals(totalRecords, records.size()); + seenKeys = new HashSet<>(); + for (GenericRecord r : records) { + assertFalse(seenKeys.contains(r.get("_row_key").toString())); + seenKeys.add(r.get("_row_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + + //RT Input Format Read - Project only non-hoodie column + reloadInputFormats(); + seenKeys = new HashSet<>(); + records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, + Arrays.asList("_row_key")); + assertEquals(totalRecords, records.size()); + for (GenericRecord r : records) { + assertFalse(seenKeys.contains(r.get("_row_key").toString())); + seenKeys.add(r.get("_row_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + } + + public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider { + + public TestFullBootstrapDataProvider(TypedProperties props, JavaSparkContext jsc) { + super(props, jsc); + } + + @Override + public JavaRDD generateInputRecordRDD(String tableName, String sourceBasePath, + List>> partitionPaths) { + String filePath = FileStatusUtils.toPath(partitionPaths.stream().flatMap(p -> p.getValue().stream()) + .findAny().get().getPath()).toString(); + ParquetFileReader reader = null; + try { + reader = ParquetFileReader.open(jsc.hadoopConfiguration(), new Path(filePath)); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + MessageType parquetSchema = reader.getFooter().getFileMetaData().getSchema(); + Schema schema = new AvroSchemaConverter().convert(parquetSchema); + return generateInputBatch(jsc, partitionPaths, schema); + } + } + + private static JavaRDD generateInputBatch(JavaSparkContext jsc, + List>> partitionPaths, Schema writerSchema) { + List> fullFilePathsWithPartition = partitionPaths.stream().flatMap(p -> p.getValue().stream() + .map(x -> Pair.of(p.getKey(), FileStatusUtils.toPath(x.getPath())))).collect(Collectors.toList()); + return jsc.parallelize(fullFilePathsWithPartition.stream().flatMap(p -> { + try { + Configuration conf = jsc.hadoopConfiguration(); + AvroReadSupport.setAvroReadSchema(conf, writerSchema); + Iterator recIterator = new ParquetReaderIterator( + AvroParquetReader.builder(p.getValue()).withConf(conf).build()); + return StreamSupport.stream(Spliterators.spliteratorUnknownSize(recIterator, 0), false).map(gr -> { + try { + String key = gr.get("_row_key").toString(); + String pPath = p.getKey(); + return new HoodieRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath, + HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + }).collect(Collectors.toList())); + } + + public static class TestRandomBootstapModeSelector extends BootstrapModeSelector { + + private int currIdx = new Random().nextInt(2); + + public TestRandomBootstapModeSelector(HoodieWriteConfig writeConfig) { + super(writeConfig); + } + + @Override + public Map> select(List>> partitions) { + List> selections = new ArrayList<>(); + partitions.stream().forEach(p -> { + final BootstrapMode mode; + if (currIdx == 0) { + mode = BootstrapMode.METADATA_ONLY; + } else { + mode = BootstrapMode.FULL_RECORD; + } + currIdx = (currIdx + 1) % 2; + selections.add(Pair.of(mode, p.getKey())); + }); + return selections.stream().collect(Collectors.groupingBy(Pair::getKey, mapping(Pair::getValue, toList()))); + } + } + + public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) { + HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, IndexType.BLOOM) + .withExternalSchemaTrasformation(true); + TypedProperties properties = new TypedProperties(); + properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key"); + properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "datestr"); + builder = builder.withProps(properties); + return builder; + } + + private static Dataset generateTestRawTripDataset(double timestamp, int numRecords, List partitionPaths, + JavaSparkContext jsc, SQLContext sqlContext) { + boolean isPartitioned = partitionPaths != null && !partitionPaths.isEmpty(); + final List records = new ArrayList<>(); + IntStream.range(0, numRecords).forEach(i -> { + String id = "" + i; + records.add(generateGenericRecord("trip_" + id, "rider_" + id, "driver_" + id, + timestamp, false, false).toString()); + }); + if (isPartitioned) { + sqlContext.udf().register("partgen", + (UDF1) (val) -> URLEncoder.encode(partitionPaths.get( + Integer.parseInt(val.split("_")[1]) % partitionPaths.size()), StandardCharsets.UTF_8.toString()), + DataTypes.StringType); + } + JavaRDD rdd = jsc.parallelize(records); + Dataset df = sqlContext.read().json(rdd); + if (isPartitioned) { + df = df.withColumn("datestr", callUDF("partgen", new Column("_row_key"))); + // Order the columns to ensure generated avro schema aligns with Hive schema + df = df.select("timestamp", "_row_key", "rider", "driver", "begin_lat", "begin_lon", + "end_lat", "end_lon", "fare", "tip_history", "_hoodie_is_deleted", "datestr"); + } else { + // Order the columns to ensure generated avro schema aligns with Hive schema + df = df.select("timestamp", "_row_key", "rider", "driver", "begin_lat", "begin_lon", + "end_lat", "end_lon", "fare", "tip_history", "_hoodie_is_deleted"); + } + return df; + } +} diff --git a/hudi-spark/src/test/resources/log4j-surefire-quiet.properties b/hudi-spark/src/test/resources/log4j-surefire-quiet.properties index b21b5d4070c41..ca0a50c84270c 100644 --- a/hudi-spark/src/test/resources/log4j-surefire-quiet.properties +++ b/hudi-spark/src/test/resources/log4j-surefire-quiet.properties @@ -17,6 +17,7 @@ ### log4j.rootLogger=WARN, CONSOLE log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR # CONSOLE is set to be a ConsoleAppender. log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender diff --git a/hudi-spark/src/test/resources/log4j-surefire.properties b/hudi-spark/src/test/resources/log4j-surefire.properties index b1ccce8f962e0..32af462093ae5 100644 --- a/hudi-spark/src/test/resources/log4j-surefire.properties +++ b/hudi-spark/src/test/resources/log4j-surefire.properties @@ -18,6 +18,7 @@ log4j.rootLogger=WARN, CONSOLE log4j.logger.org.apache=INFO log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR # A1 is set to be a ConsoleAppender. log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckPointProvider.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckPointProvider.java index 4cdc01ece6468..7fc8afb548034 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckPointProvider.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckPointProvider.java @@ -18,12 +18,12 @@ package org.apache.hudi.utilities.checkpointing; -import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.exception.HoodieException; - +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.exception.HoodieException; /** * Provide the initial checkpoint for delta streamer. @@ -51,7 +51,13 @@ public InitialCheckPointProvider(TypedProperties props) { * * @param config Hadoop configuration */ - public abstract void init(Configuration config) throws HoodieException; + public void init(Configuration config) throws HoodieException { + try { + this.fs = FileSystem.get(config); + } catch (IOException e) { + throw new HoodieException("CheckpointProvider initialization failed"); + } + } /** * Get checkpoint string recognizable for delta streamer. diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckpointFromAnotherHoodieTimelineProvider.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckpointFromAnotherHoodieTimelineProvider.java new file mode 100644 index 0000000000000..17058da7fddc6 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckpointFromAnotherHoodieTimelineProvider.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.checkpointing; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.exception.HoodieException; + +import java.io.IOException; +import java.util.Objects; + +import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY; + +/** + * This is used to set a checkpoint from latest commit of another (mirror) hudi dataset. + * Used by integration test. + */ +public class InitialCheckpointFromAnotherHoodieTimelineProvider extends InitialCheckPointProvider { + + private HoodieTableMetaClient anotherDsHoodieMetaclient; + + public InitialCheckpointFromAnotherHoodieTimelineProvider(TypedProperties props) { + super(props); + } + + @Override + public void init(Configuration config) throws HoodieException { + super.init(config); + this.anotherDsHoodieMetaclient = new HoodieTableMetaClient(config, path.toString()); + } + + @Override + public String getCheckpoint() throws HoodieException { + return anotherDsHoodieMetaclient.getCommitsTimeline().filterCompletedInstants().getReverseOrderedInstants() + .map(instant -> { + try { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(anotherDsHoodieMetaclient.getActiveTimeline().getInstantDetails(instant).get(), + HoodieCommitMetadata.class); + return commitMetadata.getMetadata(CHECKPOINT_KEY); + } catch (IOException e) { + return null; + } + }).filter(Objects::nonNull).findFirst().get(); + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java new file mode 100644 index 0000000000000..a137cac5d44d5 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.deltastreamer; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.client.HoodieWriteClient; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.hive.HiveSyncConfig; +import org.apache.hudi.hive.HiveSyncTool; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.utilities.UtilHelpers; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; + +/** + * Performs bootstrap from a non-hudi source. + */ +public class BootstrapExecutor implements Serializable { + + private static final Logger LOG = LogManager.getLogger(BootstrapExecutor.class); + + /** + * Config. + */ + private final HoodieDeltaStreamer.Config cfg; + + /** + * Schema provider that supplies the command for reading the input and writing out the target table. + */ + private transient SchemaProvider schemaProvider; + + /** + * Spark context. + */ + private transient JavaSparkContext jssc; + + /** + * Bag of properties with source, hoodie client, key generator etc. + */ + private final TypedProperties props; + + /** + * Hadoop Configuration. + */ + private final Configuration configuration; + + /** + * Bootstrap Configuration. + */ + private final HoodieWriteConfig bootstrapConfig; + + /** + * FileSystem instance. + */ + private transient FileSystem fs; + + private String bootstrapBasePath; + + /** + * Bootstrap Executor. + * @param cfg DeltaStreamer Config + * @param jssc Java Spark Context + * @param fs File System + * @param properties Bootstrap Writer Properties + * @throws IOException + */ + public BootstrapExecutor(HoodieDeltaStreamer.Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf, + TypedProperties properties) throws IOException { + this.cfg = cfg; + this.jssc = jssc; + this.fs = fs; + this.configuration = conf; + this.props = properties; + + ValidationUtils.checkArgument(properties.containsKey(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH), + HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH + " must be specified."); + this.bootstrapBasePath = properties.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH); + + // Add more defaults if full bootstrap requested + this.props.putIfAbsent(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY(), + DataSourceWriteOptions.DEFAULT_PAYLOAD_OPT_VAL()); + this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc); + HoodieWriteConfig.Builder builder = + HoodieWriteConfig.newBuilder().withPath(cfg.targetBasePath) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withInlineCompaction(false).build()) + .forTable(cfg.targetTableName) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .withAutoCommit(true) + .withProps(props); + + if (null != schemaProvider && null != schemaProvider.getTargetSchema()) { + builder = builder.withSchema(schemaProvider.getTargetSchema().toString()); + } + this.bootstrapConfig = builder.build(); + LOG.info("Created bootstrap executor with configs : " + bootstrapConfig.getProps()); + } + + /** + * Executes Bootstrap. + */ + public void execute() throws IOException { + initializeTable(); + HoodieWriteClient bootstrapClient = new HoodieWriteClient(jssc, bootstrapConfig, true); + + try { + HashMap checkpointCommitMetadata = new HashMap<>(); + checkpointCommitMetadata.put(HoodieDeltaStreamer.CHECKPOINT_KEY, cfg.checkpoint); + if (cfg.checkpoint != null) { + checkpointCommitMetadata.put(HoodieDeltaStreamer.CHECKPOINT_RESET_KEY, cfg.checkpoint); + } + bootstrapClient.bootstrap(Option.of(checkpointCommitMetadata)); + syncHive(); + } finally { + bootstrapClient.close(); + } + } + + /** + * Sync to Hive. + */ + private void syncHive() { + if (cfg.enableHiveSync) { + HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath, cfg.baseFileFormat); + LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :" + + hiveSyncConfig.jdbcUrl + ", basePath :" + cfg.targetBasePath); + new HiveSyncTool(hiveSyncConfig, new HiveConf(configuration, HiveConf.class), fs).syncHoodieTable(); + } + } + + private void initializeTable() throws IOException { + if (fs.exists(new Path(cfg.targetBasePath))) { + throw new HoodieException("target base path already exists at " + cfg.targetBasePath + + ". Cannot bootstrap data on top of an existing table"); + } + + HoodieTableMetaClient.initTableTypeWithBootstrap(new Configuration(jssc.hadoopConfiguration()), + cfg.targetBasePath, HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived", cfg.payloadClassName, + cfg.baseFileFormat, cfg.bootstrapIndexClass, bootstrapBasePath); + } + + public HoodieWriteConfig getBootstrapConfig() { + return bootstrapConfig; + } +} \ No newline at end of file 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 0db486e1118ca..ecca0d1b6c5c0 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 @@ -20,12 +20,14 @@ import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.HoodieWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -73,6 +75,8 @@ import scala.collection.JavaConversions; +import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY; +import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_RESET_KEY; import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT_PROP; import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_INSERT_PROP; import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_UPSERT_PROP; @@ -87,8 +91,6 @@ public class DeltaSync implements Serializable { private static final long serialVersionUID = 1L; private static final Logger LOG = LogManager.getLogger(DeltaSync.class); - public static final String CHECKPOINT_KEY = "deltastreamer.checkpoint.key"; - public static final String CHECKPOINT_RESET_KEY = "deltastreamer.checkpoint.reset_key"; /** * Delta Sync Config. @@ -206,14 +208,14 @@ private void refreshTimeline() throws IOException { } else { this.commitTimelineOpt = Option.empty(); HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, - cfg.tableType, cfg.targetTableName, "archived", cfg.payloadClassName, cfg.baseFileFormat); + HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived", cfg.payloadClassName, cfg.baseFileFormat); } } /** * Run one round of delta sync and return new compaction instant if one got scheduled. */ - public Pair, JavaRDD> syncOnce() throws Exception { + public Pair, JavaRDD> syncOnce() throws IOException { Pair, JavaRDD> result = null; HoodieDeltaStreamerMetrics metrics = new HoodieDeltaStreamerMetrics(getHoodieClientConfig(schemaProvider)); Timer.Context overallTimerContext = metrics.getOverallTimerContext(); @@ -249,8 +251,7 @@ public Pair, JavaRDD> syncOnce() throws Exception { * of schemaProvider, checkpointStr and hoodieRecord * @throws Exception in case of any Exception */ - public Pair>> readFromSource( - Option commitTimelineOpt) throws Exception { + public Pair>> readFromSource(Option commitTimelineOpt) throws IOException { // Retrieve the previous round checkpoints, if any Option resumeCheckpointStr = Option.empty(); if (commitTimelineOpt.isPresent()) { @@ -265,7 +266,8 @@ public Pair>> readFromSource( if (!commitMetadata.getMetadata(CHECKPOINT_KEY).isEmpty()) { resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY)); } - } else { + } else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, + HoodieTimeline.LESSER_THAN, lastCommit.get().getTimestamp())) { throw new HoodieDeltaStreamerException( "Unable to find previous checkpoint. Please double check if this table " + "was indeed built via delta streamer. Last Commit :" + lastCommit + ", Instants :" @@ -275,7 +277,7 @@ public Pair>> readFromSource( } } else { HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, - cfg.tableType, cfg.targetTableName, "archived", cfg.payloadClassName, cfg.baseFileFormat); + HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived", cfg.payloadClassName, cfg.baseFileFormat); } if (!resumeCheckpointStr.isPresent() && cfg.checkpoint != null) { @@ -338,12 +340,12 @@ public Pair>> readFromSource( } JavaRDD avroRDD = avroRDDOptional.get(); - String deleteMakrerField = props.getString(HoodieWriteConfig.DELETE_MARKER_FIELD_PROP, + String deleteMarkerField = props.getString(HoodieWriteConfig.DELETE_MARKER_FIELD_PROP, HoodieWriteConfig.DEFAULT_DELETE_MARKER_FIELD); JavaRDD records = avroRDD.map(gr -> { HoodieRecordPayload payload = DataSourceUtils.createPayload(cfg.payloadClassName, gr, - (Comparable) DataSourceUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false), - deleteMakrerField); + (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false), + deleteMarkerField); return new HoodieRecord<>(keyGenerator.getKey(gr), payload); }); @@ -360,8 +362,8 @@ public Pair>> readFromSource( * @return Option Compaction instant if one is scheduled */ private Pair, JavaRDD> writeToSink(JavaRDD records, String checkpointStr, - HoodieDeltaStreamerMetrics metrics, Timer.Context overallTimerContext) throws Exception { - + HoodieDeltaStreamerMetrics metrics, + Timer.Context overallTimerContext) { Option scheduledCompactionInstant = Option.empty(); // filter dupes if needed if (cfg.filterDupes) { @@ -476,7 +478,7 @@ private String startCommit() { /** * Sync to Hive. */ - public void syncHiveIfNeeded() throws ClassNotFoundException { + public void syncHiveIfNeeded() { if (cfg.enableHiveSync) { syncHive(); } @@ -591,3 +593,4 @@ public Option getCommitTimelineOpt() { return commitTimelineOpt; } } + 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 15624554c8b92..2f61c2ec7bf49 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 @@ -21,6 +21,7 @@ import org.apache.hudi.async.AbstractAsyncService; import org.apache.hudi.client.HoodieWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; @@ -83,11 +84,16 @@ public class HoodieDeltaStreamer implements Serializable { private static final long serialVersionUID = 1L; private static final Logger LOG = LogManager.getLogger(HoodieDeltaStreamer.class); - public static String CHECKPOINT_KEY = "deltastreamer.checkpoint.key"; + public static final String CHECKPOINT_KEY = "deltastreamer.checkpoint.key"; + public static final String CHECKPOINT_RESET_KEY = "deltastreamer.checkpoint.reset_key"; protected final transient Config cfg; - protected transient DeltaSyncService deltaSyncService; + private final TypedProperties properties; + + protected transient Option deltaSyncService; + + private final Option bootstrapExecutor; public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc) throws IOException { this(cfg, jssc, FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()), @@ -104,19 +110,27 @@ public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Con } public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf, - TypedProperties properties) throws IOException { + TypedProperties props) throws IOException { + // Resolving the properties first in a consistent way + this.properties = props != null ? props : UtilHelpers.readConfig( + FSUtils.getFs(cfg.propsFilePath, jssc.hadoopConfiguration()), + new Path(cfg.propsFilePath), cfg.configs).getConfig(); + if (cfg.initialCheckpointProvider != null && cfg.checkpoint == null) { InitialCheckPointProvider checkPointProvider = - UtilHelpers.createInitialCheckpointProvider(cfg.initialCheckpointProvider, properties); + UtilHelpers.createInitialCheckpointProvider(cfg.initialCheckpointProvider, this.properties); checkPointProvider.init(conf); cfg.checkpoint = checkPointProvider.getCheckpoint(); } this.cfg = cfg; - this.deltaSyncService = new DeltaSyncService(cfg, jssc, fs, conf, properties); + this.bootstrapExecutor = Option.ofNullable( + cfg.runBootstrap ? new BootstrapExecutor(cfg, jssc, fs, conf, this.properties) : null); + this.deltaSyncService = Option.ofNullable( + cfg.runBootstrap ? null : new DeltaSyncService(cfg, jssc, fs, conf, this.properties)); } public void shutdownGracefully() { - deltaSyncService.shutdown(false); + deltaSyncService.ifPresent(ds -> ds.shutdown(false)); } /** @@ -125,20 +139,37 @@ public void shutdownGracefully() { * @throws Exception */ public void sync() throws Exception { - if (cfg.continuousMode) { - deltaSyncService.start(this::onDeltaSyncShutdown); - deltaSyncService.waitForShutdown(); - LOG.info("Delta Sync shutting down"); + if (bootstrapExecutor.isPresent()) { + LOG.info("Performing bootstrap. Source=" + bootstrapExecutor.get().getBootstrapConfig().getBootstrapSourceBasePath()); + bootstrapExecutor.get().execute(); } else { - LOG.info("Delta Streamer running only single round"); - try { - deltaSyncService.getDeltaSync().syncOnce(); - } catch (Exception ex) { - LOG.error("Got error running delta sync once. Shutting down", ex); - throw ex; - } finally { - deltaSyncService.close(); - LOG.info("Shut down delta streamer"); + if (cfg.continuousMode) { + deltaSyncService.ifPresent(ds -> { + ds.start(this::onDeltaSyncShutdown); + try { + ds.waitForShutdown(); + } catch (Exception e) { + throw new HoodieException(e.getMessage(), e); + } + }); + LOG.info("Delta Sync shutting down"); + } else { + LOG.info("Delta Streamer running only single round"); + try { + deltaSyncService.ifPresent(ds -> { + try { + ds.getDeltaSync().syncOnce(); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }); + } catch (Exception ex) { + LOG.error("Got error running delta sync once. Shutting down", ex); + throw ex; + } finally { + deltaSyncService.ifPresent(DeltaSyncService::close); + LOG.info("Shut down delta streamer"); + } } } } @@ -149,7 +180,7 @@ public Config getConfig() { private boolean onDeltaSyncShutdown(boolean error) { LOG.info("DeltaSync shutdown. Closing write client. Error?" + error); - deltaSyncService.close(); + deltaSyncService.ifPresent(DeltaSyncService::close); return true; } @@ -181,7 +212,7 @@ public static class Config implements Serializable { public String tableType; @Parameter(names = {"--base-file-format"}, description = "File format for the base files. PARQUET (or) HFILE", required = false) - public String baseFileFormat; + public String baseFileFormat = "PARQUET"; @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 " @@ -294,6 +325,12 @@ public static class Config implements Serializable { + "Use this field only when switching source, for example, from DFS source to Kafka Source.") public String initialCheckpointProvider = null; + @Parameter(names = {"--run-bootstrap"}, description = "Run bootstrap if bootstrap index is not found") + public Boolean runBootstrap = false; + + @Parameter(names = {"--bootstrap-index-class"}, description = "subclass of BootstrapIndex") + public String bootstrapIndexClass = HFileBootstrapIndex.class.getName(); + @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; @@ -404,9 +441,7 @@ public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Config ValidationUtils.checkArgument(!cfg.filterDupes || cfg.operation != Operation.UPSERT, "'--filter-dupes' needs to be disabled when '--op' is 'UPSERT' to ensure updates are not missed."); - this.props = properties != null ? properties : UtilHelpers.readConfig( - FSUtils.getFs(cfg.propsFilePath, jssc.hadoopConfiguration()), - new Path(cfg.propsFilePath), cfg.configs).getConfig(); + this.props = properties; LOG.info("Creating delta streamer with configs : " + props.toString()); this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc); @@ -627,6 +662,6 @@ protected Pair startService() { } public DeltaSyncService getDeltaSyncService() { - return deltaSyncService; + return deltaSyncService.get(); } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index f99b738833ccb..ee8e34f95a6b9 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.table.HoodieTableConfig; @@ -70,6 +71,7 @@ import org.apache.spark.sql.api.java.UDF4; import org.apache.spark.sql.functions; import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; import org.apache.spark.streaming.kafka010.KafkaTestUtils; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -443,7 +445,7 @@ public void testTableCreation() throws Exception { } @Test - public void testBulkInsertsAndUpserts() throws Exception { + public void testBulkInsertsAndUpsertsWithBootstrap() throws Exception { String tableBasePath = dfsBasePath + "/test_table"; // Initial bulk insert @@ -469,6 +471,34 @@ public void testBulkInsertsAndUpserts() throws Exception { TestHelpers.assertCommitMetadata("00001", tableBasePath, dfs, 2); List counts = TestHelpers.countsPerCommit(tableBasePath + "/*/*.parquet", sqlContext); assertEquals(1950, counts.stream().mapToLong(entry -> entry.getLong(1)).sum()); + + // Perform bootstrap with tableBasePath as source + String bootstrapSourcePath = dfsBasePath + "/src_bootstrapped"; + sqlContext.read().format("org.apache.hudi").load(tableBasePath + "/*/*.parquet").write().format("parquet") + .save(bootstrapSourcePath); + + String newDatasetBasePath = dfsBasePath + "/test_dataset_bootstrapped"; + cfg.runBootstrap = true; + cfg.configs.add(String.format("hoodie.bootstrap.base.path=%s", bootstrapSourcePath)); + cfg.configs.add(String.format("hoodie.bootstrap.keygen.class=%s", SimpleKeyGenerator.class.getName())); + cfg.configs.add("hoodie.bootstrap.parallelism=5"); + cfg.targetBasePath = newDatasetBasePath; + new HoodieDeltaStreamer(cfg, jsc).sync(); + Dataset res = sqlContext.read().format("org.apache.hudi").load(newDatasetBasePath + "/*.parquet"); + LOG.info("Schema :"); + res.printSchema(); + + TestHelpers.assertRecordCount(1950, newDatasetBasePath + "/*.parquet", sqlContext); + res.registerTempTable("bootstrapped"); + assertEquals(1950, sqlContext.sql("select distinct _hoodie_record_key from bootstrapped").count()); + + StructField[] fields = res.schema().fields(); + assertEquals(5, fields.length); + assertEquals(HoodieRecord.COMMIT_TIME_METADATA_FIELD, fields[0].name()); + assertEquals(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, fields[1].name()); + assertEquals(HoodieRecord.RECORD_KEY_METADATA_FIELD, fields[2].name()); + assertEquals(HoodieRecord.PARTITION_PATH_METADATA_FIELD, fields[3].name()); + assertEquals(HoodieRecord.FILENAME_METADATA_FIELD, fields[4].name()); } @Test diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/source.avsc b/hudi-utilities/src/test/resources/delta-streamer-config/source.avsc index bda782e01176b..7ae44b7b35f20 100644 --- a/hudi-utilities/src/test/resources/delta-streamer-config/source.avsc +++ b/hudi-utilities/src/test/resources/delta-streamer-config/source.avsc @@ -68,8 +68,7 @@ },{ "name" : "current_ts", "type" : { - "type" : "long", - "logicalType" : "timestamp-micros" + "type" : "long" } },{ "name" : "height", diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/target.avsc b/hudi-utilities/src/test/resources/delta-streamer-config/target.avsc index a17e3ddf1b3db..815e3288bb4c5 100644 --- a/hudi-utilities/src/test/resources/delta-streamer-config/target.avsc +++ b/hudi-utilities/src/test/resources/delta-streamer-config/target.avsc @@ -68,8 +68,7 @@ },{ "name" : "current_ts", "type" : { - "type" : "long", - "logicalType" : "timestamp-micros" + "type" : "long" } }, { "name" : "height", diff --git a/hudi-utilities/src/test/resources/log4j-surefire-quiet.properties b/hudi-utilities/src/test/resources/log4j-surefire-quiet.properties index b21b5d4070c41..ca0a50c84270c 100644 --- a/hudi-utilities/src/test/resources/log4j-surefire-quiet.properties +++ b/hudi-utilities/src/test/resources/log4j-surefire-quiet.properties @@ -17,6 +17,7 @@ ### log4j.rootLogger=WARN, CONSOLE log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR # CONSOLE is set to be a ConsoleAppender. log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender diff --git a/hudi-utilities/src/test/resources/log4j-surefire.properties b/hudi-utilities/src/test/resources/log4j-surefire.properties index c03e808cca1f8..c5bdf75ae2ae3 100644 --- a/hudi-utilities/src/test/resources/log4j-surefire.properties +++ b/hudi-utilities/src/test/resources/log4j-surefire.properties @@ -18,6 +18,7 @@ log4j.rootLogger=WARN, CONSOLE log4j.logger.org.apache=INFO log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR # A1 is set to be a ConsoleAppender. log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index 9841cc7a99b12..0da29359ea9a5 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -72,9 +72,19 @@ org.objenesis:objenesis com.esotericsoftware:minlog org.apache.avro:avro + org.apache.hbase:hbase-common + org.apache.hbase:hbase-client + org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-server + org.apache.htrace:htrace-core + com.yammer.metrics:metrics-core + + com.yammer.metrics. + org.apache.hudi.com.yammer.metrics. + com.esotericsoftware.kryo. org.apache.hudi.com.esotericsoftware.kryo. @@ -91,6 +101,20 @@ org.apache.avro. org.apache.hudi.org.apache.avro. + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + + + org.apache.hadoop.hbase.util.VersionInfo + + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + false @@ -145,5 +169,37 @@ avro compile + + + org.apache.htrace + htrace-core + ${htrace.version} + compile + + + + org.apache.hbase + hbase-server + ${hbase.version} + compile + + + javax.servlet + * + + + org.codehaus.jackson + * + + + org.mortbay.jetty + * + + + tomcat + * + + + diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index c51c22b7fad90..e49a1eb8d28d6 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -72,6 +72,11 @@ com.esotericsoftware:kryo-shaded org.objenesis:objenesis com.esotericsoftware:minlog + org.apache.hbase:hbase-client + org.apache.hbase:hbase-common + org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-server + org.apache.htrace:htrace-core @@ -92,6 +97,14 @@ com.esotericsoftware.minlog. org.apache.hudi.com.esotericsoftware.minlog. + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + false diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index f4affca79044a..0f6d76efbb2fb 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -63,6 +63,7 @@ + com.yammer.metrics:metrics-core org.apache.hudi:hudi-common org.apache.hudi:hudi-client org.apache.hudi:hudi-spark_${scala.binary.version} @@ -95,9 +96,18 @@ org.apache.hive:hive-service-rpc org.apache.hive:hive-metastore org.apache.hive:hive-jdbc + org.apache.hbase:hbase-client + org.apache.hbase:hbase-common + org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-server + org.apache.htrace:htrace-core + + com.yammer.metrics. + org.apache.hudi.com.yammer.metrics. + com.beust.jcommander. org.apache.hudi.com.beust.jcommander. @@ -146,6 +156,14 @@ org.eclipse.jetty. org.apache.hudi.org.apache.jetty. + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + @@ -258,6 +276,38 @@ ${spark.bundle.hive.scope} + + org.apache.htrace + htrace-core + ${htrace.version} + compile + + + + org.apache.hbase + hbase-server + ${hbase.version} + compile + + + javax.servlet + * + + + org.codehaus.jackson + * + + + org.mortbay.jetty + * + + + tomcat + * + + + + diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index 502c42f327490..75adc7f95a08a 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -201,6 +201,10 @@ com.fasterxml.jackson.core:jackson-core com.fasterxml.jackson.core:jackson-databind org.apache.htrace:htrace-core + org.apache.hbase:hbase-common + org.apache.hbase:hbase-client + org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-server log4j:log4j diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 8cf34077c0056..9d583d32b61e2 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -73,6 +73,7 @@ org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-timeline-service + com.yammer.metrics:metrics-core com.beust:jcommander io.javalin:javalin @@ -105,9 +106,18 @@ org.apache.hive:hive-service-rpc org.apache.hive:hive-metastore org.apache.hive:hive-jdbc + org.apache.hbase:hbase-client + org.apache.hbase:hbase-common + org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-server + org.apache.htrace:htrace-core + + com.yammer.metrics. + org.apache.hudi.com.yammer.metrics. + com.beust.jcommander. org.apache.hudi.com.beust.jcommander. @@ -148,6 +158,14 @@ org.apache.commons.codec. org.apache.hudi.org.apache.commons.codec. + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + @@ -252,6 +270,39 @@ ${hive.version} ${utilities.bundle.hive.scope} + + + org.apache.htrace + htrace-core + ${htrace.version} + compile + + + + org.apache.hbase + hbase-server + ${hbase.version} + compile + + + javax.servlet + * + + + org.codehaus.jackson + * + + + org.mortbay.jetty + * + + + tomcat + * + + + + diff --git a/pom.xml b/pom.xml index 76ae34ab5f4ca..d420093f24654 100644 --- a/pom.xml +++ b/pom.xml @@ -107,6 +107,7 @@ file://${project.basedir}/src/test/resources/log4j-surefire.properties 0.12.0 9.4.15.v20190215 + 3.1.0-incubating 1.2.3 1.9.13 1.4.199