[HUDI-242] Support for RFC-12/Bootstrapping of external datasets to hudi (#1876)
- [HUDI-418] Bootstrap Index Implementation using HFile with unit-test - [HUDI-421] FileSystem View Changes to support Bootstrap with unit-tests - [HUDI-424] Implement Query Side Integration for querying tables containing bootstrap file slices - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices - [HUDI-421] Bootstrap Write Client with tests - [HUDI-425] Added HoodieDeltaStreamer support - [HUDI-899] Add a knob to change partition-path style while performing metadata bootstrap - [HUDI-900] Metadata Bootstrap Key Generator needs to handle complex keys correctly - [HUDI-424] Simplify Record reader implementation - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices - [HUDI-420] Hoodie Demo working with hive and sparkSQL. Also, Hoodie CLI working with bootstrap tables Co-authored-by: Mehrotra <uditme@amazon.com> Co-authored-by: Vinoth Chandar <vinoth@apache.org> Co-authored-by: Balaji Varadarajan <varadarb@uber.com>
This commit is contained in:
1
LICENSE
1
LICENSE
@@ -206,6 +206,7 @@
|
|||||||
This product includes code from Apache Hive.
|
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.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
|
Copyright: 2011-2019 The Apache Software Foundation
|
||||||
Home page: http://hive.apache.org/
|
Home page: http://hive.apache.org/
|
||||||
|
|||||||
@@ -19,4 +19,7 @@ connect --path /user/hive/warehouse/stock_ticks_mor
|
|||||||
compactions show all
|
compactions show all
|
||||||
compaction schedule --hoodieConfigs hoodie.compact.inline.max.delta.commits=1
|
compaction schedule --hoodieConfigs hoodie.compact.inline.max.delta.commits=1
|
||||||
compaction run --parallelism 2 --sparkMemory 1G --schemaFilePath /var/demo/config/schema.avsc --retry 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
|
||||||
|
|||||||
@@ -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_ro where symbol = 'GOOG';
|
||||||
select symbol, ts, volume, open, close from stock_ticks_mor_rt 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
|
!quit
|
||||||
|
|||||||
@@ -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_ro where symbol = 'GOOG';
|
||||||
select symbol, ts, volume, open, close from stock_ticks_mor_rt 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
|
!quit
|
||||||
|
|||||||
@@ -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}';
|
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
|
!quit
|
||||||
|
|
||||||
|
|||||||
@@ -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}';
|
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
|
!quit
|
||||||
|
|
||||||
|
|||||||
@@ -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}';
|
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
|
!quit
|
||||||
|
|
||||||
|
|||||||
@@ -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_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)
|
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)
|
System.exit(0)
|
||||||
|
|||||||
@@ -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 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)
|
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)
|
System.exit(0)
|
||||||
|
|||||||
22
docker/demo/sparksql-bootstrap-prep-source.commands
Normal file
22
docker/demo/sparksql-bootstrap-prep-source.commands
Normal file
@@ -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)
|
||||||
@@ -52,8 +52,38 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl
|
|||||||
mode(SaveMode.Overwrite).
|
mode(SaveMode.Overwrite).
|
||||||
save("/user/hive/warehouse/stock_ticks_derived_mor");
|
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_ro").show(20, false)
|
||||||
spark.sql("select count(*) from stock_ticks_derived_mor_rt").show(20, false)
|
spark.sql("select count(*) from stock_ticks_derived_mor_rt").show(20, false)
|
||||||
|
|
||||||
System.exit(0);
|
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);
|
||||||
|
|||||||
@@ -25,4 +25,7 @@ if [ -z "$CLIENT_JAR" ]; then
|
|||||||
echo "Client jar location not set, please set it in conf/hudi-env.sh"
|
echo "Client jar location not set, please set it in conf/hudi-env.sh"
|
||||||
fi
|
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 $@
|
||||||
|
|||||||
@@ -139,26 +139,6 @@
|
|||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<!-- Hoodie -->
|
<!-- Hoodie -->
|
||||||
<dependency>
|
|
||||||
<groupId>org.apache.hudi</groupId>
|
|
||||||
<artifactId>hudi-client</artifactId>
|
|
||||||
<version>${project.version}</version>
|
|
||||||
</dependency>
|
|
||||||
<dependency>
|
|
||||||
<groupId>org.apache.hudi</groupId>
|
|
||||||
<artifactId>hudi-common</artifactId>
|
|
||||||
<version>${project.version}</version>
|
|
||||||
</dependency>
|
|
||||||
<dependency>
|
|
||||||
<groupId>org.apache.hudi</groupId>
|
|
||||||
<artifactId>hudi-hive-sync</artifactId>
|
|
||||||
<version>${project.version}</version>
|
|
||||||
</dependency>
|
|
||||||
<dependency>
|
|
||||||
<groupId>org.apache.hudi</groupId>
|
|
||||||
<artifactId>hudi-utilities_${scala.binary.version}</artifactId>
|
|
||||||
<version>${project.version}</version>
|
|
||||||
</dependency>
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hudi</groupId>
|
<groupId>org.apache.hudi</groupId>
|
||||||
<artifactId>hudi-common</artifactId>
|
<artifactId>hudi-common</artifactId>
|
||||||
@@ -198,6 +178,12 @@
|
|||||||
</exclusions>
|
</exclusions>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.hudi</groupId>
|
||||||
|
<artifactId>hudi-utilities-bundle_${scala.binary.version}</artifactId>
|
||||||
|
<version>${project.version}</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
<!-- Logging -->
|
<!-- Logging -->
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>log4j</groupId>
|
<groupId>log4j</groupId>
|
||||||
|
|||||||
@@ -213,7 +213,7 @@ public class CompactionCommand implements CommandMarker {
|
|||||||
if (exitCode != 0) {
|
if (exitCode != 0) {
|
||||||
return "Failed to run compaction for " + compactionInstantTime;
|
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")
|
@CliCommand(value = "compaction run", help = "Run Compaction for given instant time")
|
||||||
|
|||||||
@@ -22,6 +22,7 @@ import org.apache.hudi.cli.HoodieCLI;
|
|||||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||||
import org.apache.hudi.cli.HoodieTableHeaderFields;
|
import org.apache.hudi.cli.HoodieTableHeaderFields;
|
||||||
import org.apache.hudi.cli.TableHeader;
|
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.FileSlice;
|
||||||
import org.apache.hudi.common.model.HoodieLogFile;
|
import org.apache.hudi.common.model.HoodieLogFile;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
@@ -239,7 +240,7 @@ public class FileSystemViewCommand implements CommandMarker {
|
|||||||
new HoodieTableMetaClient(client.getHadoopConf(), client.getBasePath(), true);
|
new HoodieTableMetaClient(client.getHadoopConf(), client.getBasePath(), true);
|
||||||
FileSystem fs = HoodieCLI.fs;
|
FileSystem fs = HoodieCLI.fs;
|
||||||
String globPath = String.format("%s/%s/*", client.getBasePath(), globRegex);
|
String globPath = String.format("%s/%s/*", client.getBasePath(), globRegex);
|
||||||
FileStatus[] statuses = fs.globStatus(new Path(globPath));
|
List<FileStatus> statuses = FSUtils.getGlobStatusExcludingMetaFolder(fs, new Path(globPath));
|
||||||
Stream<HoodieInstant> instantsStream;
|
Stream<HoodieInstant> instantsStream;
|
||||||
|
|
||||||
HoodieTimeline timeline;
|
HoodieTimeline timeline;
|
||||||
@@ -269,6 +270,6 @@ public class FileSystemViewCommand implements CommandMarker {
|
|||||||
|
|
||||||
HoodieTimeline filteredTimeline = new HoodieDefaultTimeline(instantsStream,
|
HoodieTimeline filteredTimeline = new HoodieDefaultTimeline(instantsStream,
|
||||||
(Function<HoodieInstant, Option<byte[]>> & Serializable) metaClient.getActiveTimeline()::getInstantDetails);
|
(Function<HoodieInstant, Option<byte[]>> & Serializable) metaClient.getActiveTimeline()::getInstantDetails);
|
||||||
return new HoodieTableFileSystemView(metaClient, filteredTimeline, statuses);
|
return new HoodieTableFileSystemView(metaClient, filteredTimeline, statuses.toArray(new FileStatus[0]));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,6 +22,7 @@ import org.apache.hudi.cli.HoodieCLI;
|
|||||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||||
import org.apache.hudi.cli.HoodieTableHeaderFields;
|
import org.apache.hudi.cli.HoodieTableHeaderFields;
|
||||||
import org.apache.hudi.cli.TableHeader;
|
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.HoodieLogFile;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||||
@@ -53,7 +54,6 @@ import org.springframework.stereotype.Component;
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@@ -82,7 +82,7 @@ public class HoodieLogFileCommand implements CommandMarker {
|
|||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
FileSystem fs = HoodieCLI.getTableMetaClient().getFs();
|
FileSystem fs = HoodieCLI.getTableMetaClient().getFs();
|
||||||
List<String> logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern)))
|
List<String> logFilePaths = FSUtils.getGlobStatusExcludingMetaFolder(fs, new Path(logFilePathPattern)).stream()
|
||||||
.map(status -> status.getPath().toString()).collect(Collectors.toList());
|
.map(status -> status.getPath().toString()).collect(Collectors.toList());
|
||||||
Map<String, List<Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>, Integer>>> commitCountAndMetadata =
|
Map<String, List<Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>, Integer>>> commitCountAndMetadata =
|
||||||
new HashMap<>();
|
new HashMap<>();
|
||||||
@@ -175,7 +175,7 @@ public class HoodieLogFileCommand implements CommandMarker {
|
|||||||
|
|
||||||
HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
|
HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
|
||||||
FileSystem fs = client.getFs();
|
FileSystem fs = client.getFs();
|
||||||
List<String> logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern)))
|
List<String> logFilePaths = FSUtils.getGlobStatusExcludingMetaFolder(fs, new Path(logFilePathPattern)).stream()
|
||||||
.map(status -> status.getPath().toString()).sorted(Comparator.reverseOrder())
|
.map(status -> status.getPath().toString()).sorted(Comparator.reverseOrder())
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
|||||||
@@ -118,7 +118,7 @@ public class StatsCommand implements CommandMarker {
|
|||||||
|
|
||||||
FileSystem fs = HoodieCLI.fs;
|
FileSystem fs = HoodieCLI.fs;
|
||||||
String globPath = String.format("%s/%s/*", HoodieCLI.getTableMetaClient().getBasePath(), globRegex);
|
String globPath = String.format("%s/%s/*", HoodieCLI.getTableMetaClient().getBasePath(), globRegex);
|
||||||
FileStatus[] statuses = fs.globStatus(new Path(globPath));
|
List<FileStatus> statuses = FSUtils.getGlobStatusExcludingMetaFolder(fs, new Path(globPath));
|
||||||
|
|
||||||
// max, min, #small files < 10MB, 50th, avg, 95th
|
// max, min, #small files < 10MB, 50th, avg, 95th
|
||||||
Histogram globalHistogram = new Histogram(new UniformReservoir(MAX_FILES));
|
Histogram globalHistogram = new Histogram(new UniformReservoir(MAX_FILES));
|
||||||
|
|||||||
@@ -88,8 +88,7 @@ public class TableCommand implements CommandMarker {
|
|||||||
@CliOption(key = {"archiveLogFolder"}, help = "Folder Name for storing archived timeline") String archiveFolder,
|
@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 = {"layoutVersion"}, help = "Specific Layout Version to use") Integer layoutVersion,
|
||||||
@CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "org.apache.hudi.common.model.HoodieAvroPayload",
|
@CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "org.apache.hudi.common.model.HoodieAvroPayload",
|
||||||
help = "Payload Class") final String payloadClass)
|
help = "Payload Class") final String payloadClass) throws IOException {
|
||||||
throws IOException {
|
|
||||||
|
|
||||||
boolean initialized = HoodieCLI.initConf();
|
boolean initialized = HoodieCLI.initConf();
|
||||||
HoodieCLI.initFS(initialized);
|
HoodieCLI.initFS(initialized);
|
||||||
|
|||||||
@@ -62,6 +62,7 @@ public class TestArchivedCommitsCommand extends AbstractShellIntegrationTest {
|
|||||||
// Create table and connect
|
// Create table and connect
|
||||||
String tableName = "test_table";
|
String tableName = "test_table";
|
||||||
tablePath = basePath + File.separator + tableName;
|
tablePath = basePath + File.separator + tableName;
|
||||||
|
|
||||||
new TableCommand().createTable(
|
new TableCommand().createTable(
|
||||||
tablePath, tableName,
|
tablePath, tableName,
|
||||||
"COPY_ON_WRITE", "", 1, "org.apache.hudi.common.model.HoodieAvroPayload");
|
"COPY_ON_WRITE", "", 1, "org.apache.hudi.common.model.HoodieAvroPayload");
|
||||||
|
|||||||
@@ -149,6 +149,35 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
|||||||
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Main API to run bootstrap to hudi.
|
||||||
|
*/
|
||||||
|
public void bootstrap(Option<Map<String, String>> extraMetadata) {
|
||||||
|
if (rollbackPending) {
|
||||||
|
rollBackInflightBootstrap();
|
||||||
|
}
|
||||||
|
HoodieTable<T> 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<T> table = HoodieTable.create(config, hadoopConf);
|
||||||
|
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
||||||
|
Option<String> 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.
|
* Upsert a batch of new records into Hoodie table at the supplied instantTime.
|
||||||
*
|
*
|
||||||
@@ -671,7 +700,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
|||||||
List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
|
List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
for (String commit : commits) {
|
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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -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
|
||||||
|
}
|
||||||
@@ -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<BootstrapRecordPayload> {
|
||||||
|
|
||||||
|
private final GenericRecord record;
|
||||||
|
|
||||||
|
public BootstrapRecordPayload(GenericRecord record) {
|
||||||
|
this.record = record;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BootstrapRecordPayload preCombine(BootstrapRecordPayload another) {
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) {
|
||||||
|
return Option.ofNullable(record);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Option<IndexedRecord> getInsertValue(Schema schema) {
|
||||||
|
return Option.ofNullable(record);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
@@ -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<Pair<String, List<HoodieFileStatus>>> 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<Pair<String, List<HoodieFileStatus>>> 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();
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<BootstrapFileMapping, HoodieWriteStat> getBootstrapSourceAndWriteStat() {
|
||||||
|
return Pair.of(getBootstrapSourceFileMapping(), getStat());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setBootstrapSourceFileMapping(BootstrapFileMapping sourceFileMapping) {
|
||||||
|
this.sourceFileMapping = sourceFileMapping;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<HoodieRecord> generateInputRecordRDD(String tableName,
|
||||||
|
String sourceBasePath, List<Pair<String, List<HoodieFileStatus>>> partitionPaths);
|
||||||
|
}
|
||||||
@@ -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<BootstrapMode, List<String>> select(List<Pair<String, List<HoodieFileStatus>>> partitions);
|
||||||
|
}
|
||||||
@@ -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<BootstrapMode, List<String>> select(List<Pair<String, List<HoodieFileStatus>>> 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())));
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<BootstrapMode, List<String>> select(List<Pair<String, List<HoodieFileStatus>>> partitions) {
|
||||||
|
return partitions.stream().map(p -> Pair.of(bootstrapMode, p))
|
||||||
|
.collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(x -> x.getValue().getKey(),
|
||||||
|
Collectors.toList())));
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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);
|
||||||
|
}
|
||||||
@@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<T extends GenericRecord> implements Iterator<T> {
|
||||||
|
|
||||||
|
private final Iterator<T> leftIterator;
|
||||||
|
private final Iterator<T> rightIterator;
|
||||||
|
private final Function<Pair<T,T>, T> mergeFunction;
|
||||||
|
|
||||||
|
public MergingIterator(Iterator<T> leftIterator, Iterator<T> rightIterator, Function<Pair<T,T>, 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()));
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -20,6 +20,7 @@ package org.apache.hudi.config;
|
|||||||
|
|
||||||
import org.apache.hudi.client.HoodieWriteClient;
|
import org.apache.hudi.client.HoodieWriteClient;
|
||||||
import org.apache.hudi.client.WriteStatus;
|
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.config.DefaultHoodieConfig;
|
||||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||||
@@ -129,6 +130,9 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
"_.hoodie.allow.multi.write.on.same.instant";
|
"_.hoodie.allow.multi.write.on.same.instant";
|
||||||
public static final String DEFAULT_ALLOW_MULTI_WRITE_ON_SAME_INSTANT = "false";
|
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;
|
private ConsistencyGuardConfig consistencyGuardConfig;
|
||||||
|
|
||||||
// Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled
|
// 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 final FileSystemViewStorageConfig clientSpecifiedViewStorageConfig;
|
||||||
private FileSystemViewStorageConfig viewStorageConfig;
|
private FileSystemViewStorageConfig viewStorageConfig;
|
||||||
|
|
||||||
private HoodieWriteConfig(Properties props) {
|
protected HoodieWriteConfig(Properties props) {
|
||||||
super(props);
|
super(props);
|
||||||
Properties newProps = new Properties();
|
Properties newProps = new Properties();
|
||||||
newProps.putAll(props);
|
newProps.putAll(props);
|
||||||
@@ -180,6 +184,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
return Boolean.parseBoolean(props.getProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP));
|
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() {
|
public Integer getTimelineLayoutVersion() {
|
||||||
return Integer.parseInt(props.getProperty(TIMELINE_LAYOUT_VERSION));
|
return Integer.parseInt(props.getProperty(TIMELINE_LAYOUT_VERSION));
|
||||||
}
|
}
|
||||||
@@ -675,13 +683,46 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
return props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP);
|
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 {
|
public static class Builder {
|
||||||
|
|
||||||
private final Properties props = new Properties();
|
protected final Properties props = new Properties();
|
||||||
private boolean isIndexConfigSet = false;
|
private boolean isIndexConfigSet = false;
|
||||||
private boolean isStorageConfigSet = false;
|
private boolean isStorageConfigSet = false;
|
||||||
private boolean isCompactionConfigSet = false;
|
private boolean isCompactionConfigSet = false;
|
||||||
private boolean isMetricsConfigSet = false;
|
private boolean isMetricsConfigSet = false;
|
||||||
|
private boolean isBootstrapConfigSet = false;
|
||||||
private boolean isMemoryConfigSet = false;
|
private boolean isMemoryConfigSet = false;
|
||||||
private boolean isViewConfigSet = false;
|
private boolean isViewConfigSet = false;
|
||||||
private boolean isConsistencyGuardSet = false;
|
private boolean isConsistencyGuardSet = false;
|
||||||
@@ -805,6 +846,12 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder withBootstrapConfig(HoodieBootstrapConfig bootstrapConfig) {
|
||||||
|
props.putAll(bootstrapConfig.getProps());
|
||||||
|
isBootstrapConfigSet = true;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public Builder withAutoCommit(boolean autoCommit) {
|
public Builder withAutoCommit(boolean autoCommit) {
|
||||||
props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit));
|
props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit));
|
||||||
return this;
|
return this;
|
||||||
@@ -863,7 +910,17 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
return this;
|
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
|
// Check for mandatory properties
|
||||||
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM, DEFAULT_PARALLELISM);
|
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM, DEFAULT_PARALLELISM);
|
||||||
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM), BULKINSERT_PARALLELISM,
|
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM), BULKINSERT_PARALLELISM,
|
||||||
@@ -916,6 +973,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
setDefaultOnCondition(props, !isCompactionConfigSet,
|
setDefaultOnCondition(props, !isCompactionConfigSet,
|
||||||
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
|
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
|
||||||
setDefaultOnCondition(props, !isMetricsConfigSet, HoodieMetricsConfig.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, !isMemoryConfigSet, HoodieMemoryConfig.newBuilder().fromProperties(props).build());
|
||||||
setDefaultOnCondition(props, !isViewConfigSet,
|
setDefaultOnCondition(props, !isViewConfigSet,
|
||||||
FileSystemViewStorageConfig.newBuilder().fromProperties(props).build());
|
FileSystemViewStorageConfig.newBuilder().fromProperties(props).build());
|
||||||
@@ -924,15 +983,24 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
|||||||
setDefaultOnCondition(props, !isCallbackConfigSet,
|
setDefaultOnCondition(props, !isCallbackConfigSet,
|
||||||
HoodieWriteCommitCallbackConfig.newBuilder().fromProperties(props).build());
|
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,
|
setDefaultOnCondition(props, !props.containsKey(TIMELINE_LAYOUT_VERSION), TIMELINE_LAYOUT_VERSION,
|
||||||
String.valueOf(TimelineLayoutVersion.CURR_VERSION));
|
String.valueOf(TimelineLayoutVersion.CURR_VERSION));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void validate() {
|
||||||
String layoutVersion = props.getProperty(TIMELINE_LAYOUT_VERSION);
|
String layoutVersion = props.getProperty(TIMELINE_LAYOUT_VERSION);
|
||||||
// Ensure Layout Version is good
|
// Ensure Layout Version is good
|
||||||
new TimelineLayoutVersion(Integer.parseInt(layoutVersion));
|
new TimelineLayoutVersion(Integer.parseInt(layoutVersion));
|
||||||
|
Objects.requireNonNull(props.getProperty(BASE_PATH_PROP));
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieWriteConfig build() {
|
||||||
|
setDefaults();
|
||||||
|
validate();
|
||||||
// Build WriteConfig at the end
|
// Build WriteConfig at the end
|
||||||
HoodieWriteConfig config = new HoodieWriteConfig(props);
|
HoodieWriteConfig config = new HoodieWriteConfig(props);
|
||||||
Objects.requireNonNull(config.getBasePath());
|
|
||||||
return config;
|
return config;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -165,7 +165,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
|
|||||||
private Option<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
private Option<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||||
Option recordMetadata = hoodieRecord.getData().getMetadata();
|
Option recordMetadata = hoodieRecord.getData().getMetadata();
|
||||||
try {
|
try {
|
||||||
Option<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(originalSchema);
|
Option<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(writerSchema);
|
||||||
if (avroRecord.isPresent()) {
|
if (avroRecord.isPresent()) {
|
||||||
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
||||||
avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get()));
|
avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get()));
|
||||||
@@ -212,7 +212,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
|
|||||||
private void doAppend(Map<HeaderMetadataType, String> header) {
|
private void doAppend(Map<HeaderMetadataType, String> header) {
|
||||||
try {
|
try {
|
||||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime);
|
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) {
|
if (recordList.size() > 0) {
|
||||||
writer = writer.appendBlock(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header));
|
writer = writer.appendBlock(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header));
|
||||||
recordList.clear();
|
recordList.clear();
|
||||||
|
|||||||
@@ -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 <T> HoodieRecordPayload
|
||||||
|
*/
|
||||||
|
public class HoodieBootstrapHandle<T extends HoodieRecordPayload> extends HoodieCreateHandle<T> {
|
||||||
|
|
||||||
|
public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> 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;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.io;
|
package org.apache.hudi.io;
|
||||||
|
|
||||||
|
import org.apache.avro.Schema;
|
||||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||||
import org.apache.hudi.client.WriteStatus;
|
import org.apache.hudi.client.WriteStatus;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
@@ -28,6 +29,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
|
|||||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||||
import org.apache.hudi.common.util.Option;
|
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.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieInsertException;
|
import org.apache.hudi.exception.HoodieInsertException;
|
||||||
import org.apache.hudi.io.storage.HoodieFileWriter;
|
import org.apache.hudi.io.storage.HoodieFileWriter;
|
||||||
@@ -56,8 +58,16 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWri
|
|||||||
private boolean useWriterSchema = false;
|
private boolean useWriterSchema = false;
|
||||||
|
|
||||||
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||||
String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
|
String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
|
||||||
super(config, instantTime, partitionPath, fileId, hoodieTable, sparkTaskContextSupplier);
|
this(config, instantTime, hoodieTable, partitionPath, fileId, getWriterSchemaIncludingAndExcludingMetadataPair(config),
|
||||||
|
sparkTaskContextSupplier);
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||||
|
String partitionPath, String fileId, Pair<Schema, Schema> writerSchemaIncludingAndExcludingMetadataPair,
|
||||||
|
SparkTaskContextSupplier sparkTaskContextSupplier) {
|
||||||
|
super(config, instantTime, partitionPath, fileId, hoodieTable, writerSchemaIncludingAndExcludingMetadataPair,
|
||||||
|
sparkTaskContextSupplier);
|
||||||
writeStatus.setFileId(fileId);
|
writeStatus.setFileId(fileId);
|
||||||
writeStatus.setPartitionPath(partitionPath);
|
writeStatus.setPartitionPath(partitionPath);
|
||||||
|
|
||||||
@@ -68,8 +78,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWri
|
|||||||
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
||||||
partitionMetadata.trySave(getPartitionId());
|
partitionMetadata.trySave(getPartitionId());
|
||||||
createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
|
createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
|
||||||
this.fileWriter =
|
this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writerSchemaWithMetafields, this.sparkTaskContextSupplier);
|
||||||
HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writerSchema, this.sparkTaskContextSupplier);
|
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e);
|
throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||||
}
|
}
|
||||||
@@ -132,9 +141,9 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWri
|
|||||||
while (recordIterator.hasNext()) {
|
while (recordIterator.hasNext()) {
|
||||||
HoodieRecord<T> record = recordIterator.next();
|
HoodieRecord<T> record = recordIterator.next();
|
||||||
if (useWriterSchema) {
|
if (useWriterSchema) {
|
||||||
write(record, record.getData().getInsertValue(writerSchema));
|
write(record, record.getData().getInsertValue(writerSchemaWithMetafields));
|
||||||
} else {
|
} else {
|
||||||
write(record, record.getData().getInsertValue(originalSchema));
|
write(record, record.getData().getInsertValue(writerSchema));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (IOException io) {
|
} catch (IOException io) {
|
||||||
|
|||||||
@@ -67,6 +67,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
|||||||
private long updatedRecordsWritten = 0;
|
private long updatedRecordsWritten = 0;
|
||||||
private long insertRecordsWritten = 0;
|
private long insertRecordsWritten = 0;
|
||||||
private boolean useWriterSchema;
|
private boolean useWriterSchema;
|
||||||
|
private HoodieBaseFile baseFileToMerge;
|
||||||
|
|
||||||
public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||||
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
|
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
|
||||||
@@ -88,6 +89,10 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
public Schema getWriterSchemaWithMetafields() {
|
||||||
|
return writerSchemaWithMetafields;
|
||||||
|
}
|
||||||
|
|
||||||
public Schema getWriterSchema() {
|
public Schema getWriterSchema() {
|
||||||
return writerSchema;
|
return writerSchema;
|
||||||
}
|
}
|
||||||
@@ -95,12 +100,13 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
|||||||
/**
|
/**
|
||||||
* Extract old file path, initialize StorageWriter and WriteStatus.
|
* 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);
|
LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" + fileId);
|
||||||
|
this.baseFileToMerge = baseFileToMerge;
|
||||||
this.writtenRecordKeys = new HashSet<>();
|
this.writtenRecordKeys = new HashSet<>();
|
||||||
writeStatus.setStat(new HoodieWriteStat());
|
writeStatus.setStat(new HoodieWriteStat());
|
||||||
try {
|
try {
|
||||||
String latestValidFilePath = dataFileToBeMerged.getFileName();
|
String latestValidFilePath = baseFileToMerge.getFileName();
|
||||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||||
|
|
||||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, instantTime,
|
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, instantTime,
|
||||||
@@ -126,8 +132,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
|||||||
createMarkerFile(partitionPath, newFileName);
|
createMarkerFile(partitionPath, newFileName);
|
||||||
|
|
||||||
// Create the writer for writing the new version file
|
// 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) {
|
} catch (IOException io) {
|
||||||
LOG.error("Error in update task at commit " + instantTime, io);
|
LOG.error("Error in update task at commit " + instantTime, io);
|
||||||
writeStatus.setGlobalError(io);
|
writeStatus.setGlobalError(io);
|
||||||
@@ -145,7 +150,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
|||||||
long memoryForMerge = SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps());
|
long memoryForMerge = SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps());
|
||||||
LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge);
|
LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge);
|
||||||
this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(),
|
this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(),
|
||||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(originalSchema));
|
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(writerSchema));
|
||||||
} catch (IOException io) {
|
} catch (IOException io) {
|
||||||
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
||||||
}
|
}
|
||||||
@@ -216,7 +221,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
|||||||
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key));
|
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key));
|
||||||
try {
|
try {
|
||||||
Option<IndexedRecord> combinedAvroRecord =
|
Option<IndexedRecord> combinedAvroRecord =
|
||||||
hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchema : originalSchema);
|
hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchemaWithMetafields : writerSchema);
|
||||||
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
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
|
* 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 class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
|||||||
fileWriter.writeAvro(key, oldRecord);
|
fileWriter.writeAvro(key, oldRecord);
|
||||||
} catch (ClassCastException e) {
|
} catch (ClassCastException e) {
|
||||||
LOG.error("Schema mismatch when rewriting old record " + oldRecord + " from file " + getOldFilePath()
|
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);
|
throw new HoodieUpsertException(errMsg, e);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.error("Failed to merge old record into new file for key " + key + " from old file " + getOldFilePath()
|
LOG.error("Failed to merge old record into new file for key " + key + " from old file " + getOldFilePath()
|
||||||
@@ -262,9 +267,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
|||||||
HoodieRecord<T> hoodieRecord = newRecordsItr.next();
|
HoodieRecord<T> hoodieRecord = newRecordsItr.next();
|
||||||
if (!writtenRecordKeys.contains(hoodieRecord.getRecordKey())) {
|
if (!writtenRecordKeys.contains(hoodieRecord.getRecordKey())) {
|
||||||
if (useWriterSchema) {
|
if (useWriterSchema) {
|
||||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
|
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchemaWithMetafields));
|
||||||
} else {
|
} else {
|
||||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(originalSchema));
|
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
|
||||||
}
|
}
|
||||||
insertRecordsWritten++;
|
insertRecordsWritten++;
|
||||||
}
|
}
|
||||||
@@ -312,4 +317,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
|
|||||||
public IOType getIOType() {
|
public IOType getIOType() {
|
||||||
return IOType.MERGE;
|
return IOType.MERGE;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public HoodieBaseFile baseFileForMerge() {
|
||||||
|
return baseFileToMerge;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
|
|||||||
import org.apache.hudi.common.util.HoodieTimer;
|
import org.apache.hudi.common.util.HoodieTimer;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.common.util.ReflectionUtils;
|
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.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
import org.apache.hudi.io.storage.HoodieFileWriter;
|
import org.apache.hudi.io.storage.HoodieFileWriter;
|
||||||
@@ -51,8 +52,8 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
|
|||||||
|
|
||||||
private static final Logger LOG = LogManager.getLogger(HoodieWriteHandle.class);
|
private static final Logger LOG = LogManager.getLogger(HoodieWriteHandle.class);
|
||||||
|
|
||||||
protected final Schema originalSchema;
|
|
||||||
protected final Schema writerSchema;
|
protected final Schema writerSchema;
|
||||||
|
protected final Schema writerSchemaWithMetafields;
|
||||||
protected HoodieTimer timer;
|
protected HoodieTimer timer;
|
||||||
protected final WriteStatus writeStatus;
|
protected final WriteStatus writeStatus;
|
||||||
protected final String partitionPath;
|
protected final String partitionPath;
|
||||||
@@ -62,11 +63,18 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
|
|||||||
|
|
||||||
public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath,
|
public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath,
|
||||||
String fileId, HoodieTable<T> hoodieTable, SparkTaskContextSupplier sparkTaskContextSupplier) {
|
String fileId, HoodieTable<T> hoodieTable, SparkTaskContextSupplier sparkTaskContextSupplier) {
|
||||||
|
this(config, instantTime, partitionPath, fileId, hoodieTable,
|
||||||
|
getWriterSchemaIncludingAndExcludingMetadataPair(config), sparkTaskContextSupplier);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, String fileId,
|
||||||
|
HoodieTable<T> hoodieTable, Pair<Schema, Schema> writerSchemaIncludingAndExcludingMetadataPair,
|
||||||
|
SparkTaskContextSupplier sparkTaskContextSupplier) {
|
||||||
super(config, instantTime, hoodieTable);
|
super(config, instantTime, hoodieTable);
|
||||||
this.partitionPath = partitionPath;
|
this.partitionPath = partitionPath;
|
||||||
this.fileId = fileId;
|
this.fileId = fileId;
|
||||||
this.originalSchema = new Schema.Parser().parse(config.getSchema());
|
this.writerSchema = writerSchemaIncludingAndExcludingMetadataPair.getKey();
|
||||||
this.writerSchema = HoodieAvroUtils.createHoodieWriteSchema(originalSchema);
|
this.writerSchemaWithMetafields = writerSchemaIncludingAndExcludingMetadataPair.getValue();
|
||||||
this.timer = new HoodieTimer().startTimer();
|
this.timer = new HoodieTimer().startTimer();
|
||||||
this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(),
|
this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(),
|
||||||
!hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction());
|
!hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction());
|
||||||
@@ -74,6 +82,19 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
|
|||||||
this.writeToken = makeWriteToken();
|
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<Schema, Schema> 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.
|
* Generate a write token based on the currently running spark task and its place in the spark dag.
|
||||||
*/
|
*/
|
||||||
@@ -103,8 +124,8 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
|
|||||||
markerFiles.create(partitionPath, dataFileName, getIOType());
|
markerFiles.create(partitionPath, dataFileName, getIOType());
|
||||||
}
|
}
|
||||||
|
|
||||||
public Schema getWriterSchema() {
|
public Schema getWriterSchemaWithMetafields() {
|
||||||
return writerSchema;
|
return writerSchemaWithMetafields;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -142,7 +163,7 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
|
|||||||
* Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
|
* Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
|
||||||
*/
|
*/
|
||||||
protected GenericRecord rewriteRecord(GenericRecord record) {
|
protected GenericRecord rewriteRecord(GenericRecord record) {
|
||||||
return HoodieAvroUtils.rewriteRecord(record, writerSchema);
|
return HoodieAvroUtils.rewriteRecord(record, writerSchemaWithMetafields);
|
||||||
}
|
}
|
||||||
|
|
||||||
public abstract WriteStatus close();
|
public abstract WriteStatus close();
|
||||||
|
|||||||
@@ -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<String> 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<String> getPartitionPathFields() {
|
||||||
|
throw new IllegalStateException("This method is expected to be overridden by subclasses");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final List<String> 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());
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<String> 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<String> 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;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieKey;
|
|||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
|
* Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
|
||||||
@@ -40,4 +41,14 @@ public abstract class KeyGenerator implements Serializable {
|
|||||||
* Generate a Hoodie Key out of provided generic record.
|
* Generate a Hoodie Key out of provided generic record.
|
||||||
*/
|
*/
|
||||||
public abstract HoodieKey getKey(GenericRecord record);
|
public abstract HoodieKey getKey(GenericRecord record);
|
||||||
}
|
|
||||||
|
/**
|
||||||
|
* 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<String> getRecordKeyFieldNames() {
|
||||||
|
throw new UnsupportedOperationException("Bootstrap not supported for key generator. "
|
||||||
|
+ "Please override this method in your custom key generator.");
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -18,9 +18,6 @@
|
|||||||
|
|
||||||
package org.apache.hudi.table;
|
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.HoodieCleanMetadata;
|
||||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||||
@@ -33,30 +30,32 @@ import org.apache.hudi.common.model.HoodieRecord;
|
|||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
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.Option;
|
||||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
|
||||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
|
||||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||||
import org.apache.hudi.exception.HoodieUpsertException;
|
import org.apache.hudi.exception.HoodieUpsertException;
|
||||||
import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
|
|
||||||
import org.apache.hudi.io.HoodieCreateHandle;
|
import org.apache.hudi.io.HoodieCreateHandle;
|
||||||
import org.apache.hudi.io.HoodieMergeHandle;
|
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.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.BulkInsertCommitActionExecutor;
|
||||||
import org.apache.hudi.table.action.commit.BulkInsertPreppedCommitActionExecutor;
|
import org.apache.hudi.table.action.commit.BulkInsertPreppedCommitActionExecutor;
|
||||||
import org.apache.hudi.table.action.commit.DeleteCommitActionExecutor;
|
import org.apache.hudi.table.action.commit.DeleteCommitActionExecutor;
|
||||||
import org.apache.hudi.table.action.commit.InsertCommitActionExecutor;
|
import org.apache.hudi.table.action.commit.InsertCommitActionExecutor;
|
||||||
import org.apache.hudi.table.action.commit.InsertPreppedCommitActionExecutor;
|
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.UpsertCommitActionExecutor;
|
||||||
import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor;
|
import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor;
|
||||||
import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor;
|
import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor;
|
||||||
import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
|
import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
|
||||||
import org.apache.hudi.table.action.savepoint.SavepointActionExecutor;
|
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.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
@@ -97,7 +96,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
@Override
|
@Override
|
||||||
public HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieRecord<T>> records,
|
public HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieRecord<T>> records,
|
||||||
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
||||||
return new BulkInsertCommitActionExecutor<>(jsc, config,
|
return new BulkInsertCommitActionExecutor(jsc, config,
|
||||||
this, instantTime, records, userDefinedBulkInsertPartitioner).execute();
|
this, instantTime, records, userDefinedBulkInsertPartitioner).execute();
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -121,7 +120,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
@Override
|
@Override
|
||||||
public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime,
|
public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime,
|
||||||
JavaRDD<HoodieRecord<T>> preppedRecords, Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
JavaRDD<HoodieRecord<T>> preppedRecords, Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
||||||
return new BulkInsertPreppedCommitActionExecutor<>(jsc, config,
|
return new BulkInsertPreppedCommitActionExecutor(jsc, config,
|
||||||
this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
|
this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -135,6 +134,16 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
|
throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option<Map<String, String>> 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<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
|
public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
|
||||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
|
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
|
||||||
// these are updates
|
// these are updates
|
||||||
@@ -148,25 +157,10 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
throw new HoodieUpsertException(
|
throw new HoodieUpsertException(
|
||||||
"Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
|
"Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
|
||||||
} else {
|
} else {
|
||||||
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
MergeHelper.runMerge(this, upsertHandle);
|
||||||
HoodieFileReader<IndexedRecord> 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();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
// TODO(vc): This needs to be revisited
|
// TODO(vc): This needs to be revisited
|
||||||
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
|
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
|
||||||
LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
|
LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
|
||||||
|
|||||||
@@ -28,10 +28,14 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
|
|||||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
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.Option;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieIOException;
|
import org.apache.hudi.exception.HoodieIOException;
|
||||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
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.compact.RunCompactionActionExecutor;
|
||||||
import org.apache.hudi.table.action.deltacommit.BulkInsertDeltaCommitActionExecutor;
|
import org.apache.hudi.table.action.deltacommit.BulkInsertDeltaCommitActionExecutor;
|
||||||
import org.apache.hudi.table.action.deltacommit.BulkInsertPreppedDeltaCommitActionExecutor;
|
import org.apache.hudi.table.action.deltacommit.BulkInsertPreppedDeltaCommitActionExecutor;
|
||||||
@@ -84,7 +88,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
@Override
|
@Override
|
||||||
public HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieRecord<T>> records,
|
public HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieRecord<T>> records,
|
||||||
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
||||||
return new BulkInsertDeltaCommitActionExecutor<>(jsc, config,
|
return new BulkInsertDeltaCommitActionExecutor(jsc, config,
|
||||||
this, instantTime, records, userDefinedBulkInsertPartitioner).execute();
|
this, instantTime, records, userDefinedBulkInsertPartitioner).execute();
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -108,7 +112,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
@Override
|
@Override
|
||||||
public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime,
|
public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime,
|
||||||
JavaRDD<HoodieRecord<T>> preppedRecords, Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
JavaRDD<HoodieRecord<T>> preppedRecords, Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
||||||
return new BulkInsertPreppedDeltaCommitActionExecutor<>(jsc, config,
|
return new BulkInsertPreppedDeltaCommitActionExecutor(jsc, config,
|
||||||
this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
|
this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -125,6 +129,16 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
return compactionExecutor.execute();
|
return compactionExecutor.execute();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option<Map<String, String>> 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
|
@Override
|
||||||
public HoodieRollbackMetadata rollback(JavaSparkContext jsc,
|
public HoodieRollbackMetadata rollback(JavaSparkContext jsc,
|
||||||
String rollbackInstantTime,
|
String rollbackInstantTime,
|
||||||
|
|||||||
@@ -60,6 +60,7 @@ import org.apache.hudi.exception.HoodieInsertException;
|
|||||||
import org.apache.hudi.exception.HoodieUpsertException;
|
import org.apache.hudi.exception.HoodieUpsertException;
|
||||||
import org.apache.hudi.index.HoodieIndex;
|
import org.apache.hudi.index.HoodieIndex;
|
||||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||||
|
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
@@ -331,6 +332,20 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
|||||||
public abstract HoodieWriteMetadata compact(JavaSparkContext jsc,
|
public abstract HoodieWriteMetadata compact(JavaSparkContext jsc,
|
||||||
String compactionInstantTime);
|
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<Map<String, String>> 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.
|
* Executes a new clean action.
|
||||||
*
|
*
|
||||||
|
|||||||
@@ -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<T extends HoodieRecordPayload<T>>
|
||||||
|
extends BaseCommitActionExecutor<T, HoodieBootstrapWriteMetadata> {
|
||||||
|
|
||||||
|
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<Map<String, String>> 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<HoodieInstant> 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<BootstrapMode, List<Pair<String, List<HoodieFileStatus>>>> partitionSelections = listAndProcessSourcePartitions();
|
||||||
|
|
||||||
|
// First run metadata bootstrap which will auto commit
|
||||||
|
Option<HoodieWriteMetadata> metadataResult = metadataBootstrap(partitionSelections.get(BootstrapMode.METADATA_ONLY));
|
||||||
|
// if there are full bootstrap to be performed, perform that too
|
||||||
|
Option<HoodieWriteMetadata> 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<HoodieWriteMetadata> metadataBootstrap(List<Pair<String, List<HoodieFileStatus>>> 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<BootstrapWriteStatus> bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList);
|
||||||
|
|
||||||
|
HoodieWriteMetadata result = new HoodieWriteMetadata();
|
||||||
|
updateIndexAndCommitIfNeeded(bootstrapWriteStatuses.map(w -> w), result);
|
||||||
|
return Option.of(result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void commit(Option<Map<String, String>> 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<String, List<Pair<BootstrapFileMapping, HoodieWriteStat>>> 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<HoodieWriteMetadata> fullBootstrap(List<Pair<String, List<HoodieFileStatus>>> 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<HoodieRecord> 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<T> getBulkInsertActionExecutor(JavaRDD<HoodieRecord> 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<GenericRecord, HoodieRecord, Void> wrapper = null;
|
||||||
|
try (ParquetReader<IndexedRecord> reader =
|
||||||
|
AvroParquetReader.<IndexedRecord>builder(sourceFilePath).withConf(table.getHadoopConf()).build()) {
|
||||||
|
wrapper = new SparkBoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void>(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<BootstrapMode, List<Pair<String, List<HoodieFileStatus>>>> listAndProcessSourcePartitions() throws IOException {
|
||||||
|
List<Pair<String, List<HoodieFileStatus>>> 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<BootstrapMode, List<String>> result = selector.select(folders);
|
||||||
|
Map<String, List<HoodieFileStatus>> 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<BootstrapWriteStatus> runMetadataBootstrap(List<Pair<String, List<HoodieFileStatus>>> 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<Pair<String, Pair<String, HoodieFileStatus>>> 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<List<WriteStatus>> handleInsert(String idPfx, Iterator<HoodieRecord<T>> recordItr) {
|
||||||
|
throw new UnsupportedOperationException("Should not called in bootstrap code path");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||||
|
throw new UnsupportedOperationException("Should not called in bootstrap code path");
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<T extends HoodieRecordPayload<T>>
|
||||||
|
extends BootstrapCommitActionExecutor<T> {
|
||||||
|
|
||||||
|
public BootstrapDeltaCommitActionExecutor(JavaSparkContext jsc,
|
||||||
|
HoodieWriteConfig config, HoodieTable<?> table,
|
||||||
|
Option<Map<String, String>> extraMetadata) {
|
||||||
|
super(jsc, config, table, extraMetadata);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected CommitActionExecutor<T> getBulkInsertActionExecutor(JavaRDD<HoodieRecord> inputRecordsRDD) {
|
||||||
|
return new BulkInsertDeltaCommitActionExecutor(jsc, new HoodieWriteConfig.Builder().withProps(config.getProps())
|
||||||
|
.withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
|
||||||
|
inputRecordsRDD, extraMetadata);
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<HoodieRecord, Void> {
|
||||||
|
|
||||||
|
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;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<Pair<String, List<HoodieFileStatus>>> getAllLeafFoldersWithFiles(FileSystem fs, String basePathStr,
|
||||||
|
PathFilter filePathFilter) throws IOException {
|
||||||
|
final Path basePath = new Path(basePathStr);
|
||||||
|
final Map<Integer, List<String>> levelToPartitions = new HashMap<>();
|
||||||
|
final Map<String, List<HoodieFileStatus>> partitionToFiles = new HashMap<>();
|
||||||
|
FSUtils.processFiles(fs, basePathStr, (status) -> {
|
||||||
|
if (status.isFile() && filePathFilter.accept(status.getPath())) {
|
||||||
|
String relativePath = FSUtils.getRelativePartitionPath(basePath, status.getPath().getParent());
|
||||||
|
List<HoodieFileStatus> statusList = partitionToFiles.get(relativePath);
|
||||||
|
if (null == statusList) {
|
||||||
|
Integer level = (int) relativePath.chars().filter(ch -> ch == '/').count();
|
||||||
|
List<String> 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<>();
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<HoodieWriteMetadata> metadataBootstrapResult;
|
||||||
|
private final Option<HoodieWriteMetadata> fullBootstrapResult;
|
||||||
|
|
||||||
|
public HoodieBootstrapWriteMetadata(Option<HoodieWriteMetadata> metadataBootstrapResult,
|
||||||
|
Option<HoodieWriteMetadata> fullBootstrapResult) {
|
||||||
|
this.metadataBootstrapResult = metadataBootstrapResult;
|
||||||
|
this.fullBootstrapResult = fullBootstrapResult;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Option<HoodieWriteMetadata> getMetadataBootstrapResult() {
|
||||||
|
return metadataBootstrapResult;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Option<HoodieWriteMetadata> getFullBootstrapResult() {
|
||||||
|
return fullBootstrapResult;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -18,6 +18,13 @@
|
|||||||
|
|
||||||
package org.apache.hudi.table.action.commit;
|
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.SparkTaskContextSupplier;
|
||||||
import org.apache.hudi.client.WriteStatus;
|
import org.apache.hudi.client.WriteStatus;
|
||||||
import org.apache.hudi.client.utils.SparkConfigUtils;
|
import org.apache.hudi.client.utils.SparkConfigUtils;
|
||||||
@@ -38,43 +45,31 @@ import org.apache.hudi.table.HoodieTable;
|
|||||||
import org.apache.hudi.table.WorkloadProfile;
|
import org.apache.hudi.table.WorkloadProfile;
|
||||||
import org.apache.hudi.table.WorkloadStat;
|
import org.apache.hudi.table.WorkloadStat;
|
||||||
import org.apache.hudi.table.action.BaseActionExecutor;
|
import org.apache.hudi.table.action.BaseActionExecutor;
|
||||||
|
|
||||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.Partitioner;
|
import org.apache.spark.Partitioner;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.storage.StorageLevel;
|
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;
|
import scala.Tuple2;
|
||||||
|
|
||||||
public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>, R>
|
||||||
extends BaseActionExecutor<HoodieWriteMetadata> {
|
extends BaseActionExecutor<R> {
|
||||||
|
|
||||||
private static final Logger LOG = LogManager.getLogger(BaseCommitActionExecutor.class);
|
private static final Logger LOG = LogManager.getLogger(BaseCommitActionExecutor.class);
|
||||||
|
|
||||||
|
protected final Option<Map<String, String>> extraMetadata;
|
||||||
private final WriteOperationType operationType;
|
private final WriteOperationType operationType;
|
||||||
protected final SparkTaskContextSupplier sparkTaskContextSupplier = new SparkTaskContextSupplier();
|
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,
|
public BaseCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config,
|
||||||
HoodieTable table, String instantTime, WriteOperationType operationType,
|
HoodieTable table, String instantTime, WriteOperationType operationType,
|
||||||
JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
|
Option<Map<String, String>> extraMetadata) {
|
||||||
super(jsc, config, table, instantTime);
|
super(jsc, config, table, instantTime);
|
||||||
this.operationType = operationType;
|
this.operationType = operationType;
|
||||||
|
this.extraMetadata = extraMetadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieWriteMetadata execute(JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
|
public HoodieWriteMetadata execute(JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
|
||||||
@@ -173,13 +168,17 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
|||||||
protected void commitOnAutoCommit(HoodieWriteMetadata result) {
|
protected void commitOnAutoCommit(HoodieWriteMetadata result) {
|
||||||
if (config.shouldAutoCommit()) {
|
if (config.shouldAutoCommit()) {
|
||||||
LOG.info("Auto commit enabled: Committing " + instantTime);
|
LOG.info("Auto commit enabled: Committing " + instantTime);
|
||||||
commit(Option.empty(), result);
|
commit(extraMetadata, result);
|
||||||
} else {
|
} else {
|
||||||
LOG.info("Auto commit disabled for " + instantTime);
|
LOG.info("Auto commit disabled for " + instantTime);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata result) {
|
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata result) {
|
||||||
|
commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect());
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata result, List<HoodieWriteStat> stats) {
|
||||||
String actionType = table.getMetaClient().getCommitActionType();
|
String actionType = table.getMetaClient().getCommitActionType();
|
||||||
LOG.info("Committing " + instantTime + ", action Type " + actionType);
|
LOG.info("Committing " + instantTime + ", action Type " + actionType);
|
||||||
// Create a Hoodie table which encapsulated the commits and files visible
|
// Create a Hoodie table which encapsulated the commits and files visible
|
||||||
@@ -189,7 +188,6 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
|||||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||||
|
|
||||||
result.setCommitted(true);
|
result.setCommitted(true);
|
||||||
List<HoodieWriteStat> stats = result.getWriteStatuses().map(WriteStatus::getStat).collect();
|
|
||||||
stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat));
|
stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat));
|
||||||
result.setWriteStats(stats);
|
result.setWriteStats(stats);
|
||||||
|
|
||||||
@@ -200,7 +198,7 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
|||||||
if (extraMetadata.isPresent()) {
|
if (extraMetadata.isPresent()) {
|
||||||
extraMetadata.get().forEach(metadata::addMetadata);
|
extraMetadata.get().forEach(metadata::addMetadata);
|
||||||
}
|
}
|
||||||
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, config.getSchema());
|
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit());
|
||||||
metadata.setOperationType(operationType);
|
metadata.setOperationType(operationType);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
@@ -229,6 +227,13 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* By default, return the writer schema in Write Config for storing in commit.
|
||||||
|
*/
|
||||||
|
protected String getSchemaToStoreInCommit() {
|
||||||
|
return config.getSchema();
|
||||||
|
}
|
||||||
|
|
||||||
protected boolean isWorkloadProfileNeeded() {
|
protected boolean isWorkloadProfileNeeded() {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.table.action.commit;
|
package org.apache.hudi.table.action.commit;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||||
import org.apache.hudi.common.model.WriteOperationType;
|
import org.apache.hudi.common.model.WriteOperationType;
|
||||||
@@ -31,30 +32,34 @@ import org.apache.hudi.table.action.HoodieWriteMetadata;
|
|||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
|
||||||
public class BulkInsertCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
public class BulkInsertCommitActionExecutor<T extends HoodieRecordPayload<T>> extends CommitActionExecutor<T> {
|
||||||
extends CommitActionExecutor<T> {
|
|
||||||
|
|
||||||
private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
||||||
private final Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner;
|
private final Option<BulkInsertPartitioner<T>> bulkInsertPartitioner;
|
||||||
|
|
||||||
public BulkInsertCommitActionExecutor(JavaSparkContext jsc,
|
public BulkInsertCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
|
||||||
HoodieWriteConfig config, HoodieTable table,
|
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
|
||||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
|
Option<BulkInsertPartitioner<T>> bulkInsertPartitioner) {
|
||||||
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
this(jsc, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty());
|
||||||
super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT);
|
}
|
||||||
|
|
||||||
|
public BulkInsertCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
|
||||||
|
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
|
||||||
|
Option<BulkInsertPartitioner<T>> bulkInsertPartitioner,
|
||||||
|
Option<Map<String, String>> extraMetadata) {
|
||||||
|
super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata);
|
||||||
this.inputRecordsRDD = inputRecordsRDD;
|
this.inputRecordsRDD = inputRecordsRDD;
|
||||||
this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner;
|
this.bulkInsertPartitioner = bulkInsertPartitioner;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public HoodieWriteMetadata execute() {
|
public HoodieWriteMetadata execute() {
|
||||||
try {
|
try {
|
||||||
return BulkInsertHelper.bulkInsert(inputRecordsRDD, instantTime, (HoodieTable<T>) table, config,
|
return BulkInsertHelper.bulkInsert(inputRecordsRDD, instantTime, (HoodieTable<T>) table, config,
|
||||||
this, true, userDefinedBulkInsertPartitioner);
|
this, true, bulkInsertPartitioner);
|
||||||
|
} catch (HoodieInsertException ie) {
|
||||||
|
throw ie;
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
if (e instanceof HoodieInsertException) {
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
|
throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -39,11 +39,10 @@ import java.util.stream.IntStream;
|
|||||||
|
|
||||||
public class BulkInsertHelper<T extends HoodieRecordPayload<T>> {
|
public class BulkInsertHelper<T extends HoodieRecordPayload<T>> {
|
||||||
|
|
||||||
public static <T extends HoodieRecordPayload<T>> HoodieWriteMetadata bulkInsert(
|
public static <T extends HoodieRecordPayload<T>> HoodieWriteMetadata bulkInsert(JavaRDD<HoodieRecord<T>> inputRecords, String instantTime,
|
||||||
JavaRDD<HoodieRecord<T>> inputRecords, String instantTime,
|
HoodieTable<T> table, HoodieWriteConfig config,
|
||||||
HoodieTable<T> table, HoodieWriteConfig config,
|
CommitActionExecutor<T> executor, boolean performDedupe,
|
||||||
CommitActionExecutor<T> executor, boolean performDedupe,
|
Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner) {
|
||||||
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
|
||||||
HoodieWriteMetadata result = new HoodieWriteMetadata();
|
HoodieWriteMetadata result = new HoodieWriteMetadata();
|
||||||
|
|
||||||
// De-dupe/merge if needed
|
// De-dupe/merge if needed
|
||||||
|
|||||||
@@ -35,12 +35,12 @@ public class BulkInsertPreppedCommitActionExecutor<T extends HoodieRecordPayload
|
|||||||
extends CommitActionExecutor<T> {
|
extends CommitActionExecutor<T> {
|
||||||
|
|
||||||
private final JavaRDD<HoodieRecord<T>> preppedInputRecordRdd;
|
private final JavaRDD<HoodieRecord<T>> preppedInputRecordRdd;
|
||||||
private final Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner;
|
private final Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner;
|
||||||
|
|
||||||
public BulkInsertPreppedCommitActionExecutor(JavaSparkContext jsc,
|
public BulkInsertPreppedCommitActionExecutor(JavaSparkContext jsc,
|
||||||
HoodieWriteConfig config, HoodieTable table,
|
HoodieWriteConfig config, HoodieTable table,
|
||||||
String instantTime, JavaRDD<HoodieRecord<T>> preppedInputRecordRdd,
|
String instantTime, JavaRDD<HoodieRecord<T>> preppedInputRecordRdd,
|
||||||
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
|
Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner) {
|
||||||
super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT);
|
super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT);
|
||||||
this.preppedInputRecordRdd = preppedInputRecordRdd;
|
this.preppedInputRecordRdd = preppedInputRecordRdd;
|
||||||
this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner;
|
this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner;
|
||||||
|
|||||||
@@ -23,21 +23,15 @@ import org.apache.hudi.common.model.HoodieBaseFile;
|
|||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||||
import org.apache.hudi.common.model.WriteOperationType;
|
import org.apache.hudi.common.model.WriteOperationType;
|
||||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
|
||||||
import org.apache.hudi.exception.HoodieUpsertException;
|
import org.apache.hudi.exception.HoodieUpsertException;
|
||||||
import org.apache.hudi.execution.LazyInsertIterable;
|
import org.apache.hudi.execution.LazyInsertIterable;
|
||||||
import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
|
|
||||||
import org.apache.hudi.io.HoodieMergeHandle;
|
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.HoodieTable;
|
||||||
import org.apache.hudi.table.WorkloadProfile;
|
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.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.Partitioner;
|
import org.apache.spark.Partitioner;
|
||||||
@@ -50,14 +44,19 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public abstract class CommitActionExecutor<T extends HoodieRecordPayload<T>>
|
public abstract class CommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||||
extends BaseCommitActionExecutor<T> {
|
extends BaseCommitActionExecutor<T, HoodieWriteMetadata> {
|
||||||
|
|
||||||
private static final Logger LOG = LogManager.getLogger(CommitActionExecutor.class);
|
private static final Logger LOG = LogManager.getLogger(CommitActionExecutor.class);
|
||||||
|
|
||||||
public CommitActionExecutor(JavaSparkContext jsc,
|
public CommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
|
||||||
HoodieWriteConfig config, HoodieTable table,
|
String instantTime, WriteOperationType operationType) {
|
||||||
String instantTime, WriteOperationType operationType) {
|
this(jsc, config, table, instantTime, operationType, Option.empty());
|
||||||
super(jsc, config, table, instantTime, operationType);
|
}
|
||||||
|
|
||||||
|
public CommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
|
||||||
|
String instantTime, WriteOperationType operationType,
|
||||||
|
Option<Map<String, String>> extraMetadata) {
|
||||||
|
super(jsc, config, table, instantTime, operationType, extraMetadata);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -87,22 +86,7 @@ public abstract class CommitActionExecutor<T extends HoodieRecordPayload<T>>
|
|||||||
throw new HoodieUpsertException(
|
throw new HoodieUpsertException(
|
||||||
"Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
|
"Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
|
||||||
} else {
|
} else {
|
||||||
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
MergeHelper.runMerge(table, upsertHandle);
|
||||||
try {
|
|
||||||
HoodieFileReader<IndexedRecord> 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();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO(vc): This needs to be revisited
|
// TODO(vc): This needs to be revisited
|
||||||
@@ -147,29 +131,4 @@ public abstract class CommitActionExecutor<T extends HoodieRecordPayload<T>>
|
|||||||
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
|
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
|
||||||
return getUpsertPartitioner(profile);
|
return getUpsertPartitioner(profile);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Consumer that dequeues records from queue and sends to Merge Handle.
|
|
||||||
*/
|
|
||||||
private static class UpdateHandler extends BoundedInMemoryQueueConsumer<GenericRecord, Void> {
|
|
||||||
|
|
||||||
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;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -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 <T>
|
||||||
|
* @throws IOException in case of error
|
||||||
|
*/
|
||||||
|
public static <T extends HoodieRecordPayload<T>> void runMerge(HoodieTable<T> table, HoodieMergeHandle<T> upsertHandle) throws IOException {
|
||||||
|
final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation();
|
||||||
|
Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
|
||||||
|
HoodieBaseFile baseFile = upsertHandle.baseFileForMerge();
|
||||||
|
|
||||||
|
final GenericDatumWriter<GenericRecord> gWriter;
|
||||||
|
final GenericDatumReader<GenericRecord> 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<GenericRecord, GenericRecord, Void> wrapper = null;
|
||||||
|
HoodieFileReader<GenericRecord> reader = HoodieFileReaderFactory.<T, GenericRecord>getFileReader(cfgForHoodieFile, upsertHandle.getOldFilePath());
|
||||||
|
try {
|
||||||
|
final Iterator<GenericRecord> readerIterator;
|
||||||
|
if (baseFile.getBootstrapBaseFile().isPresent()) {
|
||||||
|
readerIterator = getMergingIterator(table, upsertHandle, baseFile, reader, readSchema, externalSchemaTransformation);
|
||||||
|
} else {
|
||||||
|
readerIterator = reader.getRecordIterator(readSchema);
|
||||||
|
}
|
||||||
|
|
||||||
|
ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
|
||||||
|
ThreadLocal<BinaryDecoder> 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<GenericRecord> gReader, GenericDatumWriter<GenericRecord> gWriter,
|
||||||
|
ThreadLocal<BinaryEncoder> encoderCache, ThreadLocal<BinaryDecoder> 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 <T extends HoodieRecordPayload<T>> Iterator<GenericRecord> getMergingIterator(HoodieTable<T> table, HoodieMergeHandle<T> mergeHandle,
|
||||||
|
HoodieBaseFile baseFile, HoodieFileReader<GenericRecord> reader,
|
||||||
|
Schema readSchema, boolean externalSchemaTransformation) throws IOException {
|
||||||
|
Path externalFilePath = new Path(baseFile.getBootstrapBaseFile().get().getPath());
|
||||||
|
Configuration bootstrapFileConfig = new Configuration(table.getHadoopConf());
|
||||||
|
HoodieFileReader<GenericRecord> bootstrapReader = HoodieFileReaderFactory.<T, GenericRecord>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<GenericRecord, Void> {
|
||||||
|
|
||||||
|
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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -56,22 +56,23 @@ public class ScheduleCompactionActionExecutor extends BaseActionExecutor<Option<
|
|||||||
|
|
||||||
private HoodieCompactionPlan scheduleCompaction() {
|
private HoodieCompactionPlan scheduleCompaction() {
|
||||||
LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
|
LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||||
Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
|
Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
|
||||||
String deltaCommitsSinceTs = "0";
|
.filterCompletedInstants().lastInstant();
|
||||||
|
String lastCompactionTs = "0";
|
||||||
if (lastCompaction.isPresent()) {
|
if (lastCompaction.isPresent()) {
|
||||||
deltaCommitsSinceTs = lastCompaction.get().getTimestamp();
|
lastCompactionTs = lastCompaction.get().getTimestamp();
|
||||||
}
|
}
|
||||||
|
|
||||||
int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
|
int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
|
||||||
.findInstantsAfter(deltaCommitsSinceTs, Integer.MAX_VALUE).countInstants();
|
.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
|
||||||
if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
|
if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
|
||||||
LOG.info("Not scheduling compaction as only " + 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());
|
+ config.getInlineCompactDeltaCommitMax());
|
||||||
return new HoodieCompactionPlan();
|
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();
|
HoodieMergeOnReadTableCompactor compactor = new HoodieMergeOnReadTableCompactor();
|
||||||
try {
|
try {
|
||||||
return compactor.generateCompactionPlan(jsc, table, config, instantTime,
|
return compactor.generateCompactionPlan(jsc, table, config, instantTime,
|
||||||
|
|||||||
@@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.table.action.deltacommit;
|
package org.apache.hudi.table.action.deltacommit;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||||
import org.apache.hudi.common.model.WriteOperationType;
|
import org.apache.hudi.common.model.WriteOperationType;
|
||||||
@@ -36,13 +37,19 @@ public class BulkInsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T
|
|||||||
extends DeltaCommitActionExecutor<T> {
|
extends DeltaCommitActionExecutor<T> {
|
||||||
|
|
||||||
private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
||||||
private final Option<BulkInsertPartitioner> bulkInsertPartitioner;
|
private final Option<BulkInsertPartitioner<T>> bulkInsertPartitioner;
|
||||||
|
|
||||||
public BulkInsertDeltaCommitActionExecutor(JavaSparkContext jsc,
|
public BulkInsertDeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
|
||||||
HoodieWriteConfig config, HoodieTable table,
|
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
|
||||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
|
Option<BulkInsertPartitioner<T>> bulkInsertPartitioner) {
|
||||||
Option<BulkInsertPartitioner> bulkInsertPartitioner) {
|
this(jsc, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty());
|
||||||
super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT);
|
}
|
||||||
|
|
||||||
|
public BulkInsertDeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
|
||||||
|
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
|
||||||
|
Option<BulkInsertPartitioner<T>> bulkInsertPartitioner,
|
||||||
|
Option<Map<String, String>> extraMetadata) {
|
||||||
|
super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata);
|
||||||
this.inputRecordsRDD = inputRecordsRDD;
|
this.inputRecordsRDD = inputRecordsRDD;
|
||||||
this.bulkInsertPartitioner = bulkInsertPartitioner;
|
this.bulkInsertPartitioner = bulkInsertPartitioner;
|
||||||
}
|
}
|
||||||
@@ -52,10 +59,9 @@ public class BulkInsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T
|
|||||||
try {
|
try {
|
||||||
return BulkInsertHelper.bulkInsert(inputRecordsRDD, instantTime, (HoodieTable<T>) table, config,
|
return BulkInsertHelper.bulkInsert(inputRecordsRDD, instantTime, (HoodieTable<T>) table, config,
|
||||||
this, true, bulkInsertPartitioner);
|
this, true, bulkInsertPartitioner);
|
||||||
|
} catch (HoodieInsertException ie) {
|
||||||
|
throw ie;
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
if (e instanceof HoodieInsertException) {
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
|
throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -36,12 +36,12 @@ public class BulkInsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPa
|
|||||||
extends DeltaCommitActionExecutor<T> {
|
extends DeltaCommitActionExecutor<T> {
|
||||||
|
|
||||||
private final JavaRDD<HoodieRecord<T>> preppedInputRecordRdd;
|
private final JavaRDD<HoodieRecord<T>> preppedInputRecordRdd;
|
||||||
private final Option<BulkInsertPartitioner> bulkInsertPartitioner;
|
private final Option<BulkInsertPartitioner<T>> bulkInsertPartitioner;
|
||||||
|
|
||||||
public BulkInsertPreppedDeltaCommitActionExecutor(JavaSparkContext jsc,
|
public BulkInsertPreppedDeltaCommitActionExecutor(JavaSparkContext jsc,
|
||||||
HoodieWriteConfig config, HoodieTable table,
|
HoodieWriteConfig config, HoodieTable table,
|
||||||
String instantTime, JavaRDD<HoodieRecord<T>> preppedInputRecordRdd,
|
String instantTime, JavaRDD<HoodieRecord<T>> preppedInputRecordRdd,
|
||||||
Option<BulkInsertPartitioner> bulkInsertPartitioner) {
|
Option<BulkInsertPartitioner<T>> bulkInsertPartitioner) {
|
||||||
super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT);
|
super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT);
|
||||||
this.preppedInputRecordRdd = preppedInputRecordRdd;
|
this.preppedInputRecordRdd = preppedInputRecordRdd;
|
||||||
this.bulkInsertPartitioner = bulkInsertPartitioner;
|
this.bulkInsertPartitioner = bulkInsertPartitioner;
|
||||||
|
|||||||
@@ -18,10 +18,12 @@
|
|||||||
|
|
||||||
package org.apache.hudi.table.action.deltacommit;
|
package org.apache.hudi.table.action.deltacommit;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
import org.apache.hudi.client.WriteStatus;
|
import org.apache.hudi.client.WriteStatus;
|
||||||
import org.apache.hudi.common.model.HoodieRecord;
|
import org.apache.hudi.common.model.HoodieRecord;
|
||||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||||
import org.apache.hudi.common.model.WriteOperationType;
|
import org.apache.hudi.common.model.WriteOperationType;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
import org.apache.hudi.exception.HoodieUpsertException;
|
import org.apache.hudi.exception.HoodieUpsertException;
|
||||||
import org.apache.hudi.execution.LazyInsertIterable;
|
import org.apache.hudi.execution.LazyInsertIterable;
|
||||||
@@ -48,10 +50,15 @@ public abstract class DeltaCommitActionExecutor<T extends HoodieRecordPayload<T>
|
|||||||
// UpsertPartitioner for MergeOnRead table type
|
// UpsertPartitioner for MergeOnRead table type
|
||||||
private UpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner;
|
private UpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner;
|
||||||
|
|
||||||
public DeltaCommitActionExecutor(JavaSparkContext jsc,
|
public DeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
|
||||||
HoodieWriteConfig config, HoodieTable table,
|
String instantTime, WriteOperationType operationType) {
|
||||||
String instantTime, WriteOperationType operationType) {
|
this(jsc, config, table, instantTime, operationType, Option.empty());
|
||||||
super(jsc, config, table, instantTime, operationType);
|
}
|
||||||
|
|
||||||
|
public DeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
|
||||||
|
String instantTime, WriteOperationType operationType,
|
||||||
|
Option<Map<String, String>> extraMetadata) {
|
||||||
|
super(jsc, config, table, instantTime, operationType, extraMetadata);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|||||||
@@ -20,6 +20,7 @@ package org.apache.hudi.table.action.rollback;
|
|||||||
|
|
||||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||||
import org.apache.hudi.common.HoodieRollbackStat;
|
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.fs.FSUtils;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
@@ -224,4 +225,11 @@ public abstract class BaseRollbackActionExecutor extends BaseActionExecutor<Hood
|
|||||||
LOG.warn("Rollback finished without deleting inflight instant file. Instant=" + instantToBeDeleted);
|
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();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -82,6 +82,9 @@ public class CopyOnWriteRollbackActionExecutor extends BaseRollbackActionExecuto
|
|||||||
LOG.info("Clean out all base files generated for commit: " + resolvedInstant);
|
LOG.info("Clean out all base files generated for commit: " + resolvedInstant);
|
||||||
stats = getRollbackStrategy().execute(resolvedInstant);
|
stats = getRollbackStrategy().execute(resolvedInstant);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
dropBootstrapIndexIfNeeded(instantToRollback);
|
||||||
|
|
||||||
// Delete Inflight instant if enabled
|
// Delete Inflight instant if enabled
|
||||||
deleteInflightAndRequestedInstant(deleteInstants, activeTimeline, resolvedInstant);
|
deleteInflightAndRequestedInstant(deleteInstants, activeTimeline, resolvedInstant);
|
||||||
LOG.info("Time(in ms) taken to finish rollback " + rollbackTimer.endTimer());
|
LOG.info("Time(in ms) taken to finish rollback " + rollbackTimer.endTimer());
|
||||||
|
|||||||
@@ -98,6 +98,8 @@ public class MergeOnReadRollbackActionExecutor extends BaseRollbackActionExecuto
|
|||||||
allRollbackStats = getRollbackStrategy().execute(resolvedInstant);
|
allRollbackStats = getRollbackStrategy().execute(resolvedInstant);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
dropBootstrapIndexIfNeeded(resolvedInstant);
|
||||||
|
|
||||||
// Delete Inflight instants if enabled
|
// Delete Inflight instants if enabled
|
||||||
deleteInflightAndRequestedInstant(deleteInstants, table.getActiveTimeline(), resolvedInstant);
|
deleteInflightAndRequestedInstant(deleteInstants, table.getActiveTimeline(), resolvedInstant);
|
||||||
LOG.info("Time(in ms) taken to finish rollback " + rollbackTimer.endTimer());
|
LOG.info("Time(in ms) taken to finish rollback " + rollbackTimer.endTimer());
|
||||||
|
|||||||
@@ -477,8 +477,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
|||||||
.withBloomIndexUpdatePartitionPath(true)
|
.withBloomIndexUpdatePartitionPath(true)
|
||||||
.withGlobalSimpleIndexUpdatePartitionPath(true)
|
.withGlobalSimpleIndexUpdatePartitionPath(true)
|
||||||
.build()).withTimelineLayoutVersion(VERSION_0).build();
|
.build()).withTimelineLayoutVersion(VERSION_0).build();
|
||||||
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(),
|
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(),
|
||||||
metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(),
|
metaClient.getTableType(), metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(),
|
||||||
metaClient.getTableConfig().getPayloadClass(), VERSION_0);
|
metaClient.getTableConfig().getPayloadClass(), VERSION_0);
|
||||||
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||||
|
|
||||||
|
|||||||
@@ -122,7 +122,7 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
|||||||
HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config2, "101", table2,
|
HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config2, "101", table2,
|
||||||
updateRecords.iterator(), record1.getPartitionPath(), fileId, supplier);
|
updateRecords.iterator(), record1.getPartitionPath(), fileId, supplier);
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
AvroReadSupport.setAvroReadSchema(conf, mergeHandle.getWriterSchema());
|
AvroReadSupport.setAvroReadSchema(conf, mergeHandle.getWriterSchemaWithMetafields());
|
||||||
List<GenericRecord> oldRecords = ParquetUtils.readAvroRecords(conf,
|
List<GenericRecord> oldRecords = ParquetUtils.readAvroRecords(conf,
|
||||||
new Path(config2.getBasePath() + "/" + insertResult.getStat().getPath()));
|
new Path(config2.getBasePath() + "/" + insertResult.getStat().getPath()));
|
||||||
for (GenericRecord rec : oldRecords) {
|
for (GenericRecord rec : oldRecords) {
|
||||||
|
|||||||
@@ -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<String> partitionPaths = Arrays.asList("2020/05/01", "2020/05/02", "2020/05/10", "2020/05/11");
|
||||||
|
List<Pair<String, List<HoodieFileStatus>>> input = partitionPaths.stream()
|
||||||
|
.map(p -> Pair.<String, List<HoodieFileStatus>>of(p, new ArrayList<>())).collect(Collectors.toList());
|
||||||
|
String regex = "2020/05/1[0-9]";
|
||||||
|
BootstrapRegexModeSelector regexModeSelector = new BootstrapRegexModeSelector(getConfig(regex,
|
||||||
|
BootstrapMode.FULL_RECORD));
|
||||||
|
|
||||||
|
Map<BootstrapMode, List<String>> 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());
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<String> partitionPaths = Arrays.asList("2020/05/01", "2020/05/02", "2020/05/10", "2020/05/11");
|
||||||
|
List<Pair<String, List<HoodieFileStatus>>> input = partitionPaths.stream()
|
||||||
|
.map(p -> Pair.<String, List<HoodieFileStatus>>of(p, new ArrayList<>())).collect(Collectors.toList());
|
||||||
|
Map<BootstrapMode, List<String>> 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());
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<String> 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<String> 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<Pair<String, List<HoodieFileStatus>>> 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());
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -19,6 +19,7 @@
|
|||||||
package org.apache.hudi.table.action.compact.strategy;
|
package org.apache.hudi.table.action.compact.strategy;
|
||||||
|
|
||||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
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.HoodieBaseFile;
|
||||||
import org.apache.hudi.common.model.HoodieLogFile;
|
import org.apache.hudi.common.model.HoodieLogFile;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
@@ -259,7 +260,9 @@ public class TestHoodieCompactionStrategy {
|
|||||||
operations.add(new HoodieCompactionOperation(df.getCommitTime(),
|
operations.add(new HoodieCompactionOperation(df.getCommitTime(),
|
||||||
logFiles.stream().map(s -> s.getPath().toString()).collect(Collectors.toList()), df.getPath(), df.getFileId(),
|
logFiles.stream().map(s -> s.getPath().toString()).collect(Collectors.toList()), df.getPath(), df.getFileId(),
|
||||||
partitionPath,
|
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;
|
return operations;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,14 +22,17 @@ import org.apache.hudi.avro.HoodieAvroUtils;
|
|||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
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.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
|
import org.apache.avro.Schema.Field;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.avro.generic.GenericRecordBuilder;
|
import org.apache.avro.generic.GenericRecordBuilder;
|
||||||
import org.apache.hadoop.conf.Configurable;
|
import org.apache.hadoop.conf.Configurable;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
|
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.hive.serde2.ColumnProjectionUtils;
|
||||||
import org.apache.hadoop.io.ArrayWritable;
|
import org.apache.hadoop.io.ArrayWritable;
|
||||||
import org.apache.hadoop.io.Writable;
|
import org.apache.hadoop.io.Writable;
|
||||||
@@ -48,22 +51,39 @@ import java.util.stream.Collectors;
|
|||||||
* Utility methods to aid in testing MergeOnRead (workaround for HoodieReadClient for MOR).
|
* Utility methods to aid in testing MergeOnRead (workaround for HoodieReadClient for MOR).
|
||||||
*/
|
*/
|
||||||
public class HoodieMergeOnReadTestUtils {
|
public class HoodieMergeOnReadTestUtils {
|
||||||
|
|
||||||
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths,
|
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths,
|
||||||
String basePath) {
|
String basePath) {
|
||||||
return getRecordsUsingInputFormat(conf, inputPaths, basePath, new JobConf(conf), true);
|
return getRecordsUsingInputFormat(conf, inputPaths, basePath, new JobConf(conf), true);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths,
|
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths,
|
||||||
String basePath,
|
String basePath, JobConf jobConf, boolean realtime) {
|
||||||
JobConf jobConf,
|
Schema schema = new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA);
|
||||||
boolean realtime) {
|
return getRecordsUsingInputFormat(conf, inputPaths, basePath, jobConf, realtime, schema,
|
||||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf, basePath);
|
HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
|
||||||
FileInputFormat inputFormat = HoodieInputFormatUtils.getInputFormat(metaClient.getTableConfig().getBaseFileFormat(),
|
}
|
||||||
realtime, jobConf);
|
|
||||||
|
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths, String basePath, JobConf jobConf, boolean realtime, Schema rawSchema,
|
||||||
|
String rawHiveColumnTypes, boolean projectCols, List<String> projectedColumns) {
|
||||||
|
|
||||||
|
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf, basePath);
|
||||||
|
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<Field> 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 -> {
|
return inputPaths.stream().map(path -> {
|
||||||
setInputPath(jobConf, path);
|
setInputPath(jobConf, path);
|
||||||
List<GenericRecord> records = new ArrayList<>();
|
List<GenericRecord> records = new ArrayList<>();
|
||||||
@@ -71,17 +91,18 @@ public class HoodieMergeOnReadTestUtils {
|
|||||||
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
|
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
|
||||||
for (InputSplit split : splits) {
|
for (InputSplit split : splits) {
|
||||||
RecordReader recordReader = inputFormat.getRecordReader(split, jobConf, null);
|
RecordReader recordReader = inputFormat.getRecordReader(split, jobConf, null);
|
||||||
Void key = (Void) recordReader.createKey();
|
Object key = recordReader.createKey();
|
||||||
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
|
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
|
||||||
while (recordReader.next(key, writable)) {
|
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,
|
// writable returns an array with [field1, field2, _hoodie_commit_time,
|
||||||
// _hoodie_commit_seqno]
|
// _hoodie_commit_seqno]
|
||||||
Writable[] values = writable.get();
|
Writable[] values = writable.get();
|
||||||
assert schema.getFields().size() <= values.length;
|
schema.getFields().stream()
|
||||||
schema.getFields().forEach(field -> {
|
.filter(f -> !projectCols || projectedColumns.contains(f.name()))
|
||||||
newRecord.set(field, values[field.pos()]);
|
.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());
|
records.add(newRecord.build());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -95,29 +116,40 @@ public class HoodieMergeOnReadTestUtils {
|
|||||||
}).orElse(new ArrayList<>());
|
}).orElse(new ArrayList<>());
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf jobConf, Schema schema,
|
private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf jobConf, Schema schema, String hiveColumnTypes, boolean projectCols, List<String> projectedCols) {
|
||||||
String basePath) {
|
|
||||||
List<Schema.Field> fields = schema.getFields();
|
List<Schema.Field> fields = schema.getFields();
|
||||||
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
|
final List<String> projectedColNames;
|
||||||
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
|
if (!projectCols) {
|
||||||
Configuration conf = HoodieTestUtils.getDefaultHadoopConf();
|
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(","));
|
.map(Schema.Field::name).collect(Collectors.joining(","));
|
||||||
hiveColumnNames = hiveColumnNames + ",datestr";
|
hiveColumnNames = hiveColumnNames + ",datestr";
|
||||||
|
|
||||||
String hiveColumnTypes = HoodieAvroUtils.addMetadataColumnTypes(HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES);
|
Configuration conf = HoodieTestUtils.getDefaultHadoopConf();
|
||||||
hiveColumnTypes = hiveColumnTypes + ",string";
|
String hiveColumnTypesWithDatestr = hiveColumnTypes + ",string";
|
||||||
jobConf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames);
|
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_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");
|
jobConf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "datestr");
|
||||||
conf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames);
|
conf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames);
|
||||||
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
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_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
|
// Hoodie Input formats are also configurable
|
||||||
Configurable configurable = (Configurable)inputFormat;
|
Configurable configurable = (Configurable)inputFormat;
|
||||||
|
|||||||
@@ -17,6 +17,7 @@
|
|||||||
###
|
###
|
||||||
log4j.rootLogger=WARN, CONSOLE
|
log4j.rootLogger=WARN, CONSOLE
|
||||||
log4j.logger.org.apache.hudi=DEBUG
|
log4j.logger.org.apache.hudi=DEBUG
|
||||||
|
log4j.logger.org.apache.hadoop.hbase=ERROR
|
||||||
|
|
||||||
# CONSOLE is set to be a ConsoleAppender.
|
# CONSOLE is set to be a ConsoleAppender.
|
||||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||||
|
|||||||
@@ -18,6 +18,7 @@
|
|||||||
log4j.rootLogger=WARN, CONSOLE
|
log4j.rootLogger=WARN, CONSOLE
|
||||||
log4j.logger.org.apache=INFO
|
log4j.logger.org.apache=INFO
|
||||||
log4j.logger.org.apache.hudi=DEBUG
|
log4j.logger.org.apache.hudi=DEBUG
|
||||||
|
log4j.logger.org.apache.hadoop.hbase=ERROR
|
||||||
|
|
||||||
# A1 is set to be a ConsoleAppender.
|
# A1 is set to be a ConsoleAppender.
|
||||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||||
|
|||||||
@@ -75,6 +75,12 @@
|
|||||||
<import>${basedir}/src/main/avro/HoodieRollbackMetadata.avsc</import>
|
<import>${basedir}/src/main/avro/HoodieRollbackMetadata.avsc</import>
|
||||||
<import>${basedir}/src/main/avro/HoodieRestoreMetadata.avsc</import>
|
<import>${basedir}/src/main/avro/HoodieRestoreMetadata.avsc</import>
|
||||||
<import>${basedir}/src/main/avro/HoodieArchivedMetaEntry.avsc</import>
|
<import>${basedir}/src/main/avro/HoodieArchivedMetaEntry.avsc</import>
|
||||||
|
<import>${basedir}/src/main/avro/HoodiePath.avsc</import>
|
||||||
|
<import>${basedir}/src/main/avro/HoodieFSPermission.avsc</import>
|
||||||
|
<import>${basedir}/src/main/avro/HoodieFileStatus.avsc</import>
|
||||||
|
<import>${basedir}/src/main/avro/HoodieBootstrapSourceFilePartitionInfo.avsc</import>
|
||||||
|
<import>${basedir}/src/main/avro/HoodieBootstrapIndexInfo.avsc</import>
|
||||||
|
<import>${basedir}/src/main/avro/HoodieBootstrapMetadata.avsc</import>
|
||||||
</imports>
|
</imports>
|
||||||
</configuration>
|
</configuration>
|
||||||
</plugin>
|
</plugin>
|
||||||
@@ -197,12 +203,31 @@
|
|||||||
<version>${hbase.version}</version>
|
<version>${hbase.version}</version>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hbase</groupId>
|
<groupId>org.apache.hbase</groupId>
|
||||||
<artifactId>hbase-server</artifactId>
|
<artifactId>hbase-server</artifactId>
|
||||||
<version>${hbase.version}</version>
|
<version>${hbase.version}</version>
|
||||||
<scope>test</scope>
|
<!-- Unfortunately, HFile is packaged ONLY under hbase-server -->
|
||||||
|
<scope>compile</scope>
|
||||||
|
<exclusions>
|
||||||
|
<exclusion>
|
||||||
|
<groupId>javax.servlet</groupId>
|
||||||
|
<artifactId>*</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
<exclusion>
|
||||||
|
<groupId>org.codehaus.jackson</groupId>
|
||||||
|
<artifactId>*</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
<exclusion>
|
||||||
|
<groupId>org.mortbay.jetty</groupId>
|
||||||
|
<artifactId>*</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
<exclusion>
|
||||||
|
<groupId>tomcat</groupId>
|
||||||
|
<artifactId>*</artifactId>
|
||||||
|
</exclusion>
|
||||||
|
</exclusions>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
</dependencies>
|
</dependencies>
|
||||||
</project>
|
</project>
|
||||||
|
|||||||
@@ -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
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
44
hudi-common/src/main/avro/HoodieBootstrapIndexInfo.avsc
Normal file
44
hudi-common/src/main/avro/HoodieBootstrapIndexInfo.avsc
Normal file
@@ -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
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
@@ -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
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
@@ -61,6 +61,11 @@
|
|||||||
"values":"double"
|
"values":"double"
|
||||||
}],
|
}],
|
||||||
"default": null
|
"default": null
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"bootstrapFilePath",
|
||||||
|
"type":["null", "string"],
|
||||||
|
"default": null
|
||||||
}
|
}
|
||||||
]
|
]
|
||||||
}
|
}
|
||||||
|
|||||||
49
hudi-common/src/main/avro/HoodieFSPermission.avsc
Normal file
49
hudi-common/src/main/avro/HoodieFSPermission.avsc
Normal file
@@ -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"
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
84
hudi-common/src/main/avro/HoodieFileStatus.avsc
Normal file
84
hudi-common/src/main/avro/HoodieFileStatus.avsc
Normal file
@@ -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
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
34
hudi-common/src/main/avro/HoodiePath.avsc
Normal file
34
hudi-common/src/main/avro/HoodiePath.avsc
Normal file
@@ -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
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
@@ -19,7 +19,13 @@
|
|||||||
package org.apache.hudi.avro;
|
package org.apache.hudi.avro;
|
||||||
|
|
||||||
import org.apache.avro.JsonProperties;
|
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.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.HoodieIOException;
|
||||||
import org.apache.hudi.exception.SchemaCompatabilityException;
|
import org.apache.hudi.exception.SchemaCompatabilityException;
|
||||||
|
|
||||||
@@ -64,10 +70,10 @@ public class HoodieAvroUtils {
|
|||||||
private static ThreadLocal<BinaryDecoder> reuseDecoder = ThreadLocal.withInitial(() -> null);
|
private static ThreadLocal<BinaryDecoder> reuseDecoder = ThreadLocal.withInitial(() -> null);
|
||||||
|
|
||||||
// All metadata fields are optional strings.
|
// 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)));
|
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.
|
* Convert a given avro record to bytes.
|
||||||
@@ -251,6 +257,17 @@ public class HoodieAvroUtils {
|
|||||||
return record;
|
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
|
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the old
|
||||||
* schema.
|
* schema.
|
||||||
@@ -326,4 +343,105 @@ public class HoodieAvroUtils {
|
|||||||
throw new HoodieIOException("IOException while decompressing text", e);
|
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<String> fieldNames) {
|
||||||
|
Map<String, Field> schemaFieldsMap = originalSchema.getFields().stream()
|
||||||
|
.map(r -> Pair.of(r.name().toLowerCase(), r)).collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
|
||||||
|
List<Schema.Field> 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();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
@@ -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<String> getIndexedPartitionPaths();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return list file-ids indexed.
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
public abstract List<String> getIndexedFileIds();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Lookup bootstrap index by partition.
|
||||||
|
* @param partition Partition to lookup
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
public abstract List<BootstrapFileMapping> getSourceFileMappingForPartition(String partition);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Lookup Bootstrap index by file group ids.
|
||||||
|
* @param ids File Group Ids
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
public abstract Map<HoodieFileGroupId, BootstrapFileMapping> getSourceFileMappingForFileIds(
|
||||||
|
List<HoodieFileGroupId> 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<BootstrapFileMapping> 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)));
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<String> getIndexedPartitionPaths() {
|
||||||
|
HFileScanner scanner = partitionIndexReader().getScanner(true, true);
|
||||||
|
return getAllKeys(scanner);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> getIndexedFileIds() {
|
||||||
|
HFileScanner scanner = fileIdIndexReader().getScanner(true, true);
|
||||||
|
return getAllKeys(scanner);
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<String> getAllKeys(HFileScanner scanner) {
|
||||||
|
List<String> 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<BootstrapFileMapping> 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<HoodieFileGroupId, BootstrapFileMapping> getSourceFileMappingForFileIds(
|
||||||
|
List<HoodieFileGroupId> ids) {
|
||||||
|
Map<HoodieFileGroupId, BootstrapFileMapping> result = new HashMap<>();
|
||||||
|
// Arrange input Keys in sorted order for 1 pass scan
|
||||||
|
List<HoodieFileGroupId> 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<String, List<BootstrapFileMapping>> 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<BootstrapFileMapping> 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<byte[]> 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<BootstrapFileMapping> bootstrapFileMappings) {
|
||||||
|
sourceFileMappings.put(partitionPath, bootstrapFileMappings);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void finish() {
|
||||||
|
// Sort and write
|
||||||
|
List<String> 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 {
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.common.fs;
|
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.HoodieFileFormat;
|
||||||
import org.apache.hudi.common.model.HoodieLogFile;
|
import org.apache.hudi.common.model.HoodieLogFile;
|
||||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||||
@@ -45,6 +46,7 @@ import java.io.File;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
@@ -52,6 +54,7 @@ import java.util.UUID;
|
|||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
import java.util.regex.Matcher;
|
import java.util.regex.Matcher;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -114,6 +117,10 @@ public class FSUtils {
|
|||||||
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, fileExtension);
|
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) {
|
public static String maskWithoutFileId(String instantTime, int taskPartitionId) {
|
||||||
return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieFileFormat.PARQUET.getFileExtension());
|
return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieFileFormat.PARQUET.getFileExtension());
|
||||||
}
|
}
|
||||||
@@ -225,8 +232,8 @@ public class FSUtils {
|
|||||||
|
|
||||||
public static String getFileExtension(String fullName) {
|
public static String getFileExtension(String fullName) {
|
||||||
Objects.requireNonNull(fullName);
|
Objects.requireNonNull(fullName);
|
||||||
String fileName = (new File(fullName)).getName();
|
String fileName = new File(fullName).getName();
|
||||||
int dotIndex = fileName.indexOf('.');
|
int dotIndex = fileName.lastIndexOf('.');
|
||||||
return dotIndex == -1 ? "" : fileName.substring(dotIndex);
|
return dotIndex == -1 ? "" : fileName.substring(dotIndex);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -235,10 +242,6 @@ public class FSUtils {
|
|||||||
return (path) -> !path.toString().contains(HoodieTableMetaClient.METAFOLDER_NAME);
|
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.
|
* Returns a new unique prefix for creating a file group.
|
||||||
*/
|
*/
|
||||||
@@ -516,4 +519,33 @@ public class FSUtils {
|
|||||||
return returnConf;
|
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<FileStatus> 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());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -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 + '}';
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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
|
||||||
|
+ '}';
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -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<BootstrapFileMapping> {
|
||||||
|
|
||||||
|
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;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -44,18 +44,20 @@ public class CompactionOperation implements Serializable {
|
|||||||
private Option<String> dataFileName;
|
private Option<String> dataFileName;
|
||||||
private HoodieFileGroupId id;
|
private HoodieFileGroupId id;
|
||||||
private Map<String, Double> metrics;
|
private Map<String, Double> metrics;
|
||||||
|
private Option<String> bootstrapFilePath;
|
||||||
|
|
||||||
// Only for serialization/de-serialization
|
// Only for serialization/de-serialization
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public CompactionOperation() {}
|
public CompactionOperation() {}
|
||||||
|
|
||||||
public CompactionOperation(String fileId, String partitionPath, String baseInstantTime,
|
public CompactionOperation(String fileId, String partitionPath, String baseInstantTime,
|
||||||
Option<String> dataFileCommitTime, List<String> deltaFileNames, Option<String> dataFileName,
|
Option<String> dataFileCommitTime, List<String> deltaFileNames, Option<String> dataFileName,
|
||||||
Map<String, Double> metrics) {
|
Option<String> bootstrapFilePath, Map<String, Double> metrics) {
|
||||||
this.baseInstantTime = baseInstantTime;
|
this.baseInstantTime = baseInstantTime;
|
||||||
this.dataFileCommitTime = dataFileCommitTime;
|
this.dataFileCommitTime = dataFileCommitTime;
|
||||||
this.deltaFileNames = deltaFileNames;
|
this.deltaFileNames = deltaFileNames;
|
||||||
this.dataFileName = dataFileName;
|
this.dataFileName = dataFileName;
|
||||||
|
this.bootstrapFilePath = bootstrapFilePath;
|
||||||
this.id = new HoodieFileGroupId(partitionPath, fileId);
|
this.id = new HoodieFileGroupId(partitionPath, fileId);
|
||||||
this.metrics = metrics;
|
this.metrics = metrics;
|
||||||
}
|
}
|
||||||
@@ -67,14 +69,15 @@ public class CompactionOperation implements Serializable {
|
|||||||
this.dataFileName = Option.of(dataFile.get().getFileName());
|
this.dataFileName = Option.of(dataFile.get().getFileName());
|
||||||
this.id = new HoodieFileGroupId(partitionPath, dataFile.get().getFileId());
|
this.id = new HoodieFileGroupId(partitionPath, dataFile.get().getFileId());
|
||||||
this.dataFileCommitTime = Option.of(dataFile.get().getCommitTime());
|
this.dataFileCommitTime = Option.of(dataFile.get().getCommitTime());
|
||||||
|
this.bootstrapFilePath = dataFile.get().getBootstrapBaseFile().map(BaseFile::getPath);
|
||||||
} else {
|
} else {
|
||||||
assert logFiles.size() > 0;
|
assert logFiles.size() > 0;
|
||||||
this.dataFileName = Option.empty();
|
this.dataFileName = Option.empty();
|
||||||
this.baseInstantTime = FSUtils.getBaseCommitTimeFromLogPath(logFiles.get(0).getPath());
|
this.baseInstantTime = FSUtils.getBaseCommitTimeFromLogPath(logFiles.get(0).getPath());
|
||||||
this.id = new HoodieFileGroupId(partitionPath, FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath()));
|
this.id = new HoodieFileGroupId(partitionPath, FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath()));
|
||||||
this.dataFileCommitTime = Option.empty();
|
this.dataFileCommitTime = Option.empty();
|
||||||
|
this.bootstrapFilePath = Option.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
this.deltaFileNames = logFiles.stream().map(s -> s.getPath().getName()).collect(Collectors.toList());
|
this.deltaFileNames = logFiles.stream().map(s -> s.getPath().getName()).collect(Collectors.toList());
|
||||||
this.metrics = metrics;
|
this.metrics = metrics;
|
||||||
}
|
}
|
||||||
@@ -111,9 +114,17 @@ public class CompactionOperation implements Serializable {
|
|||||||
return id;
|
return id;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Option<String> getBootstrapFilePath() {
|
||||||
|
return bootstrapFilePath;
|
||||||
|
}
|
||||||
|
|
||||||
public Option<HoodieBaseFile> getBaseFile(String basePath, String partitionPath) {
|
public Option<HoodieBaseFile> getBaseFile(String basePath, String partitionPath) {
|
||||||
|
Option<BaseFile> externalBaseFile = bootstrapFilePath.map(BaseFile::new);
|
||||||
Path dirPath = FSUtils.getPartitionPath(basePath, partitionPath);
|
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 class CompactionOperation implements Serializable {
|
|||||||
op.deltaFileNames = new ArrayList<>(operation.getDeltaFilePaths());
|
op.deltaFileNames = new ArrayList<>(operation.getDeltaFilePaths());
|
||||||
op.id = new HoodieFileGroupId(operation.getPartitionPath(), operation.getFileId());
|
op.id = new HoodieFileGroupId(operation.getPartitionPath(), operation.getFileId());
|
||||||
op.metrics = operation.getMetrics() == null ? new HashMap<>() : new HashMap<>(operation.getMetrics());
|
op.metrics = operation.getMetrics() == null ? new HashMap<>() : new HashMap<>(operation.getMetrics());
|
||||||
|
op.bootstrapFilePath = Option.ofNullable(operation.getBootstrapFilePath());
|
||||||
return op;
|
return op;
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -137,7 +149,7 @@ public class CompactionOperation implements Serializable {
|
|||||||
public String toString() {
|
public String toString() {
|
||||||
return "CompactionOperation{baseInstantTime='" + baseInstantTime + '\'' + ", dataFileCommitTime="
|
return "CompactionOperation{baseInstantTime='" + baseInstantTime + '\'' + ", dataFileCommitTime="
|
||||||
+ dataFileCommitTime + ", deltaFileNames=" + deltaFileNames + ", dataFileName=" + dataFileName + ", id='" + id
|
+ dataFileCommitTime + ", deltaFileNames=" + deltaFileNames + ", dataFileName=" + dataFileName + ", id='" + id
|
||||||
+ '\'' + ", metrics=" + metrics + '}';
|
+ '\'' + ", metrics=" + metrics + ", bootstrapFilePath=" + bootstrapFilePath + '}';
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|||||||
@@ -52,6 +52,14 @@ public class FileSlice implements Serializable {
|
|||||||
*/
|
*/
|
||||||
private final TreeSet<HoodieLogFile> logFiles;
|
private final TreeSet<HoodieLogFile> 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) {
|
public FileSlice(String partitionPath, String baseInstantTime, String fileId) {
|
||||||
this(new HoodieFileGroupId(partitionPath, fileId), baseInstantTime);
|
this(new HoodieFileGroupId(partitionPath, fileId), baseInstantTime);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -19,33 +19,39 @@
|
|||||||
package org.apache.hudi.common.model;
|
package org.apache.hudi.common.model;
|
||||||
|
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
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.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 {
|
public class HoodieBaseFile extends BaseFile {
|
||||||
|
|
||||||
private static final long serialVersionUID = 1L;
|
private Option<BaseFile> bootstrapBaseFile;
|
||||||
private transient FileStatus fileStatus;
|
|
||||||
private final String fullPath;
|
public HoodieBaseFile(HoodieBaseFile dataFile) {
|
||||||
private long fileLen;
|
super(dataFile);
|
||||||
|
this.bootstrapBaseFile = dataFile.bootstrapBaseFile;
|
||||||
|
}
|
||||||
|
|
||||||
public HoodieBaseFile(FileStatus fileStatus) {
|
public HoodieBaseFile(FileStatus fileStatus) {
|
||||||
this.fileStatus = fileStatus;
|
this(fileStatus, null);
|
||||||
this.fullPath = fileStatus.getPath().toString();
|
}
|
||||||
this.fileLen = fileStatus.getLen();
|
|
||||||
|
public HoodieBaseFile(FileStatus fileStatus, BaseFile bootstrapBaseFile) {
|
||||||
|
super(fileStatus);
|
||||||
|
this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieBaseFile(String filePath) {
|
public HoodieBaseFile(String filePath) {
|
||||||
this.fileStatus = null;
|
this(filePath, null);
|
||||||
this.fullPath = filePath;
|
}
|
||||||
this.fileLen = -1;
|
|
||||||
|
public HoodieBaseFile(String filePath, BaseFile bootstrapBaseFile) {
|
||||||
|
super(filePath);
|
||||||
|
this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getFileId() {
|
public String getFileId() {
|
||||||
@@ -56,49 +62,17 @@ public class HoodieBaseFile implements Serializable {
|
|||||||
return FSUtils.getCommitTime(getFileName());
|
return FSUtils.getCommitTime(getFileName());
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getPath() {
|
public Option<BaseFile> getBootstrapBaseFile() {
|
||||||
return fullPath;
|
return bootstrapBaseFile;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getFileName() {
|
public void setBootstrapBaseFile(BaseFile bootstrapBaseFile) {
|
||||||
return new Path(fullPath).getName();
|
this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile);
|
||||||
}
|
|
||||||
|
|
||||||
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;
|
|
||||||
}
|
|
||||||
HoodieBaseFile dataFile = (HoodieBaseFile) o;
|
|
||||||
return Objects.equals(fullPath, dataFile.fullPath);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
return Objects.hash(fullPath);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "HoodieDataFile{fullPath=" + fullPath + ", fileLen=" + fileLen + '}';
|
return "HoodieBaseFile{fullPath=" + getPath() + ", fileLen=" + getFileLen()
|
||||||
|
+ ", BootstrapBaseFile=" + bootstrapBaseFile.orElse(null) + '}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,7 +22,9 @@ package org.apache.hudi.common.model;
|
|||||||
* Hoodie file format.
|
* Hoodie file format.
|
||||||
*/
|
*/
|
||||||
public enum HoodieFileFormat {
|
public enum HoodieFileFormat {
|
||||||
PARQUET(".parquet"), HOODIE_LOG(".log");
|
PARQUET(".parquet"),
|
||||||
|
HOODIE_LOG(".log"),
|
||||||
|
HFILE(".hfile");
|
||||||
|
|
||||||
private final String extension;
|
private final String extension;
|
||||||
|
|
||||||
|
|||||||
@@ -21,12 +21,14 @@ package org.apache.hudi.common.model;
|
|||||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -58,6 +60,15 @@ public class HoodieFileGroup implements Serializable {
|
|||||||
*/
|
*/
|
||||||
private final Option<HoodieInstant> lastInstant;
|
private final Option<HoodieInstant> 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) {
|
public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) {
|
||||||
this(new HoodieFileGroupId(partitionPath, id), timeline);
|
this(new HoodieFileGroupId(partitionPath, id), timeline);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -24,7 +24,7 @@ import java.util.Objects;
|
|||||||
/**
|
/**
|
||||||
* Unique ID to identify a file-group in a data-set.
|
* Unique ID to identify a file-group in a data-set.
|
||||||
*/
|
*/
|
||||||
public class HoodieFileGroupId implements Serializable {
|
public class HoodieFileGroupId implements Serializable, Comparable<HoodieFileGroupId> {
|
||||||
|
|
||||||
private final String partitionPath;
|
private final String partitionPath;
|
||||||
|
|
||||||
@@ -64,4 +64,13 @@ public class HoodieFileGroupId implements Serializable {
|
|||||||
public String toString() {
|
public String toString() {
|
||||||
return "HoodieFileGroupId{partitionPath='" + partitionPath + '\'' + ", fileId='" + fileId + '\'' + '}';
|
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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -45,6 +45,12 @@ public class HoodieLogFile implements Serializable {
|
|||||||
private final String pathStr;
|
private final String pathStr;
|
||||||
private long fileLen;
|
private long fileLen;
|
||||||
|
|
||||||
|
public HoodieLogFile(HoodieLogFile logFile) {
|
||||||
|
this.fileStatus = logFile.fileStatus;
|
||||||
|
this.pathStr = logFile.pathStr;
|
||||||
|
this.fileLen = logFile.fileLen;
|
||||||
|
}
|
||||||
|
|
||||||
public HoodieLogFile(FileStatus fileStatus) {
|
public HoodieLogFile(FileStatus fileStatus) {
|
||||||
this.fileStatus = fileStatus;
|
this.fileStatus = fileStatus;
|
||||||
this.pathStr = fileStatus.getPath().toString();
|
this.pathStr = fileStatus.getPath().toString();
|
||||||
|
|||||||
@@ -37,6 +37,7 @@ public enum WriteOperationType {
|
|||||||
BULK_INSERT_PREPPED("bulk_insert_prepped"),
|
BULK_INSERT_PREPPED("bulk_insert_prepped"),
|
||||||
// delete
|
// delete
|
||||||
DELETE("delete"),
|
DELETE("delete"),
|
||||||
|
BOOTSTRAP("bootstrap"),
|
||||||
// used for old version
|
// used for old version
|
||||||
UNKNOWN("unknown");
|
UNKNOWN("unknown");
|
||||||
|
|
||||||
|
|||||||
@@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.common.table;
|
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.HoodieFileFormat;
|
||||||
import org.apache.hudi.common.model.HoodieTableType;
|
import org.apache.hudi.common.model.HoodieTableType;
|
||||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
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_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_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_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 HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
|
||||||
public static final HoodieFileFormat DEFAULT_BASE_FILE_FORMAT = HoodieFileFormat.PARQUET;
|
public static final HoodieFileFormat DEFAULT_BASE_FILE_FORMAT = HoodieFileFormat.PARQUET;
|
||||||
public static final HoodieFileFormat DEFAULT_LOG_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
|
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_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 Integer DEFAULT_TIMELINE_LAYOUT_VERSION = TimelineLayoutVersion.VERSION_0;
|
||||||
public static final String DEFAULT_ARCHIVELOG_FOLDER = "";
|
public static final String DEFAULT_ARCHIVELOG_FOLDER = "";
|
||||||
private Properties props;
|
private Properties props;
|
||||||
@@ -131,6 +136,10 @@ public class HoodieTableConfig implements Serializable {
|
|||||||
// Use latest Version as default unless forced by client
|
// Use latest Version as default unless forced by client
|
||||||
properties.setProperty(HOODIE_TIMELINE_LAYOUT_VERSION, TimelineLayoutVersion.CURR_VERSION.toString());
|
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()));
|
properties.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@@ -161,6 +170,19 @@ public class HoodieTableConfig implements Serializable {
|
|||||||
"org.apache.hudi");
|
"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<String> getBootstrapBasePath() {
|
||||||
|
return Option.ofNullable(props.getProperty(HOODIE_BOOTSTRAP_BASE_PATH));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Read the table name.
|
* Read the table name.
|
||||||
*/
|
*/
|
||||||
|
|||||||
@@ -74,6 +74,12 @@ public class HoodieTableMetaClient implements Serializable {
|
|||||||
public static final String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp";
|
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 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_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";
|
public static final String MARKER_EXTN = ".marker";
|
||||||
|
|
||||||
private String basePath;
|
private String basePath;
|
||||||
@@ -209,6 +215,20 @@ public class HoodieTableMetaClient implements Serializable {
|
|||||||
return basePath + File.separator + AUXILIARYFOLDER_NAME;
|
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
|
* @return path where archived timeline is stored
|
||||||
*/
|
*/
|
||||||
@@ -300,44 +320,45 @@ public class HoodieTableMetaClient implements Serializable {
|
|||||||
return archivedTimeline;
|
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
|
* Helper method to initialize a table, with given basePath, tableType, name, archiveFolder, payloadClass and
|
||||||
* base file format.
|
* base file format.
|
||||||
*/
|
*/
|
||||||
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType,
|
public static HoodieTableMetaClient initTableTypeWithBootstrap(Configuration hadoopConf, String basePath, HoodieTableType tableType,
|
||||||
String tableName, String archiveLogFolder, String payloadClassName, String baseFileFormat) throws IOException {
|
String tableName, String archiveLogFolder, String payloadClassName,
|
||||||
return initTableType(hadoopConf, basePath, HoodieTableType.valueOf(tableType), tableName,
|
String baseFileFormat, String bootstrapIndexClass,
|
||||||
archiveLogFolder, payloadClassName, null, baseFileFormat);
|
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,
|
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
|
||||||
HoodieTableType tableType, String tableName, String payloadClassName) throws IOException {
|
String tableName, String payloadClassName) throws IOException {
|
||||||
return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName, null, null);
|
return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName,
|
||||||
|
null, null, null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
|
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
|
||||||
HoodieTableType tableType, String tableName,
|
String tableName, String archiveLogFolder, String payloadClassName,
|
||||||
String archiveLogFolder, String payloadClassName,
|
|
||||||
Integer timelineLayoutVersion) throws IOException {
|
Integer timelineLayoutVersion) throws IOException {
|
||||||
return initTableType(hadoopConf, basePath, tableType, tableName, archiveLogFolder, payloadClassName,
|
return initTableType(hadoopConf, basePath, tableType, tableName, archiveLogFolder, payloadClassName,
|
||||||
timelineLayoutVersion, null);
|
timelineLayoutVersion, null, null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
|
private static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
|
||||||
HoodieTableType tableType, String tableName, String archiveLogFolder, String payloadClassName,
|
String tableName, String archiveLogFolder, String payloadClassName,
|
||||||
Integer timelineLayoutVersion, String baseFileFormat) throws IOException {
|
Integer timelineLayoutVersion, String baseFileFormat,
|
||||||
|
String bootstrapIndexClass, String bootstrapBasePath) throws IOException {
|
||||||
Properties properties = new Properties();
|
Properties properties = new Properties();
|
||||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
|
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
|
||||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
|
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
|
||||||
@@ -356,6 +377,15 @@ public class HoodieTableMetaClient implements Serializable {
|
|||||||
if (null != baseFileFormat) {
|
if (null != baseFileFormat) {
|
||||||
properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, baseFileFormat.toUpperCase());
|
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);
|
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -399,6 +429,7 @@ public class HoodieTableMetaClient implements Serializable {
|
|||||||
fs.mkdirs(auxiliaryFolder);
|
fs.mkdirs(auxiliaryFolder);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
initializeBootstrapDirsIfNotExists(hadoopConf, basePath, fs);
|
||||||
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props);
|
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props);
|
||||||
// We should not use fs.getConf as this might be different from the original configuration
|
// We should not use fs.getConf as this might be different from the original configuration
|
||||||
// used to create the fs in unit tests
|
// used to create the fs in unit tests
|
||||||
@@ -407,6 +438,25 @@ public class HoodieTableMetaClient implements Serializable {
|
|||||||
return metaClient;
|
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.
|
* Helper method to scan all hoodie-instant metafiles.
|
||||||
*
|
*
|
||||||
@@ -512,7 +562,7 @@ public class HoodieTableMetaClient implements Serializable {
|
|||||||
HoodieTableMetaClient
|
HoodieTableMetaClient
|
||||||
.scanFiles(getFs(), timelinePath, path -> {
|
.scanFiles(getFs(), timelinePath, path -> {
|
||||||
// Include only the meta files with extensions that needs to be included
|
// 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);
|
return includedExtensions.contains(extension);
|
||||||
})).map(HoodieInstant::new);
|
})).map(HoodieInstant::new);
|
||||||
|
|
||||||
@@ -549,6 +599,10 @@ public class HoodieTableMetaClient implements Serializable {
|
|||||||
return sb.toString();
|
return sb.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void initializeBootstrapDirsIfNotExists() throws IOException {
|
||||||
|
initializeBootstrapDirsIfNotExists(getHadoopConf(), basePath, getFs());
|
||||||
|
}
|
||||||
|
|
||||||
public void setBasePath(String basePath) {
|
public void setBasePath(String basePath) {
|
||||||
this.basePath = basePath;
|
this.basePath = basePath;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,9 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.common.table.timeline;
|
package org.apache.hudi.common.table.timeline;
|
||||||
|
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
|
||||||
import org.apache.hudi.common.util.CollectionUtils;
|
import org.apache.hudi.common.util.CollectionUtils;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
@@ -53,6 +51,12 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
|
|||||||
return COMPARABLE_ACTIONS.getOrDefault(action, 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.
|
* Instant State.
|
||||||
*/
|
*/
|
||||||
@@ -77,7 +81,7 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
|
|||||||
public HoodieInstant(FileStatus fileStatus) {
|
public HoodieInstant(FileStatus fileStatus) {
|
||||||
// First read the instant timestamp. [==>20170101193025<==].commit
|
// First read the instant timestamp. [==>20170101193025<==].commit
|
||||||
String fileName = fileStatus.getPath().getName();
|
String fileName = fileStatus.getPath().getName();
|
||||||
String fileExtension = FSUtils.getFileExtension(fileName);
|
String fileExtension = getTimelineFileExtension(fileName);
|
||||||
timestamp = fileName.replace(fileExtension, "");
|
timestamp = fileName.replace(fileExtension, "");
|
||||||
|
|
||||||
// Next read the action for this marker
|
// Next read the action for this marker
|
||||||
|
|||||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user